From 8134e33dee9b064682a8953bf8c9b8fb32abdf8e Mon Sep 17 00:00:00 2001 From: Xuyang Date: Fri, 17 Jan 2025 11:00:18 +0800 Subject: [PATCH] [FLINK-36927][table] Introduce slicing & one stage Window Aggregate with Async State API This closes #25861 --- .../SliceAttachedWindowingStrategy.java | 4 +- .../stream/StreamExecWindowAggregate.java | 20 +- .../table/planner/plan/utils/WindowUtil.scala | 21 +- .../stream/sql/WindowAggregateITCase.scala | 857 ++++++------------ .../AsyncStateTableStreamOperator.java | 24 +- .../operators/TableStreamOperator.java | 8 +- .../AsyncStateRecordsWindowBuffer.java | 226 +++++ .../buffers/AsyncStateWindowBuffer.java | 133 +++ .../combines/AsyncStateAggCombiner.java | 147 +++ ...ractAsyncStateSliceWindowAggProcessor.java | 197 ++++ .../AbstractAsyncStateWindowAggProcessor.java | 121 +++ ...yncStateSliceSharedWindowAggProcessor.java | 207 +++++ ...cStateSliceUnsharedWindowAggProcessor.java | 79 ++ .../window/WindowAggOperatorBuilder.java | 132 ++- ...ractSliceSyncStateWindowAggProcessor.java} | 13 +- ... AbstractSyncStateWindowAggProcessor.java} | 89 +- ...iceSharedSyncStateWindowAggProcessor.java} | 9 +- ...eUnsharedSyncStateWindowAggProcessor.java} | 7 +- ...> UnsliceSyncStateWindowAggProcessor.java} | 11 +- .../processors/WindowAggProcessorBase.java | 120 +++ ...wTimeWindowDeduplicateOperatorBuilder.java | 8 +- ...eSyncStateWindowDeduplicateProcessor.java} | 11 +- .../window/WindowRankOperatorBuilder.java | 8 +- ...java => SyncStateWindowRankProcessor.java} | 10 +- .../window/async/AsyncMergeCallback.java | 57 ++ .../combines/AsyncStateRecordsCombiner.java | 69 ++ .../common/AsyncStateWindowAggOperator.java | 336 +++++++ .../tvf/common/AsyncStateWindowProcessor.java | 97 ++ .../AsyncStateSlicingWindowProcessor.java | 60 ++ .../async/tvf/state/AsyncStateKeyContext.java | 51 ++ .../async/tvf/state/WindowAsyncState.java | 28 + .../tvf/state/WindowAsyncValueState.java | 59 ++ .../tvf/common/SyncStateWindowProcessor.java | 82 ++ .../window/tvf/common/WindowAggOperator.java | 20 +- .../window/tvf/common/WindowProcessor.java | 73 +- .../window/tvf/slicing/SliceAssigner.java | 2 +- .../window/tvf/slicing/SliceAssigners.java | 81 +- .../tvf/slicing/SliceSharedAssigner.java | 25 +- ...a => SlicingSyncStateWindowProcessor.java} | 26 +- .../window/tvf/unslicing/UnsliceAssigner.java | 4 +- ...=> UnslicingSyncStateWindowProcessor.java} | 13 +- .../table/runtime/util/AsyncStateUtils.java | 32 + .../window/SlicingWindowAggOperatorTest.java | 95 +- .../UnslicingWindowAggOperatorTest.java | 80 +- .../window/WindowAggOperatorTestBase.java | 118 ++- 45 files changed, 2916 insertions(+), 954 deletions(-) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateRecordsWindowBuffer.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateWindowBuffer.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/combines/AsyncStateAggCombiner.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateSliceWindowAggProcessor.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateWindowAggProcessor.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceUnsharedWindowAggProcessor.java rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/{AbstractSliceWindowAggProcessor.java => AbstractSliceSyncStateWindowAggProcessor.java} (94%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/{AbstractWindowAggProcessor.java => AbstractSyncStateWindowAggProcessor.java} (61%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/{SliceSharedWindowAggProcessor.java => SliceSharedSyncStateWindowAggProcessor.java} (94%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/{SliceUnsharedWindowAggProcessor.java => SliceUnsharedSyncStateWindowAggProcessor.java} (90%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/{UnsliceWindowAggProcessor.java => UnsliceSyncStateWindowAggProcessor.java} (97%) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/WindowAggProcessorBase.java rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/{RowTimeWindowDeduplicateProcessor.java => RowTimeSyncStateWindowDeduplicateProcessor.java} (94%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/{WindowRankProcessor.java => SyncStateWindowRankProcessor.java} (96%) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/AsyncMergeCallback.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/combines/AsyncStateRecordsCombiner.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowAggOperator.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowProcessor.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/slicing/AsyncStateSlicingWindowProcessor.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/AsyncStateKeyContext.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncState.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncValueState.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/SyncStateWindowProcessor.java rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/{SlicingWindowProcessor.java => SlicingSyncStateWindowProcessor.java} (62%) rename flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/{UnslicingWindowProcessor.java => UnslicingSyncStateWindowProcessor.java} (70%) create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/AsyncStateUtils.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java index 1cf8e30b96139..414544105f73e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/logical/SliceAttachedWindowingStrategy.java @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.logical; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -31,7 +31,7 @@ * A windowing strategy that gets windows from input columns as window slice have been assigned and * attached to the physical columns. The window slice is usually identified by slice end timestamp. * - * @see SlicingWindowProcessor for more details about which windows can apply slicing. + * @see SlicingSyncStateWindowProcessor for more details about which windows can apply slicing. */ @JsonTypeName("SliceAttached") public class SliceAttachedWindowingStrategy extends WindowingStrategy { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java index dcf8223c4105e..78696900d78b7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java @@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; +import org.apache.flink.table.planner.codegen.EqualiserCodeGenerator; import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.logical.WindowingStrategy; @@ -39,6 +40,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList; import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.planner.plan.utils.WindowUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.planner.utils.TableConfigUtils; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; @@ -198,7 +200,7 @@ protected Transformation translateToPlanInternal( InternalTypeInfo.of(inputRowType)); final LogicalType[] accTypes = convertToLogicalTypes(aggInfoList.getAccTypes()); - final OneInputStreamOperator windowOperator = + final WindowAggOperatorBuilder windowAggOperatorBuilder = WindowAggOperatorBuilder.builder() .inputSerializer(new RowDataSerializer(inputRowType)) .shiftTimeZone(shiftTimeZone) @@ -207,8 +209,20 @@ protected Transformation translateToPlanInternal( selector.getProducedType().toSerializer()) .assigner(windowAssigner) .countStarIndex(aggInfoList.getIndexOfCountStar()) - .aggregate(generatedAggsHandler, new RowDataSerializer(accTypes)) - .build(); + .aggregate(generatedAggsHandler, new RowDataSerializer(accTypes)); + + if (WindowUtil.isAsyncStateEnabled(config, windowAssigner, aggInfoList)) { + windowAggOperatorBuilder + .enableAsyncState() + .generatedKeyEqualiser( + new EqualiserCodeGenerator( + selector.getProducedType().toRowType(), + planner.getFlinkContext().getClassLoader()) + .generateRecordEqualiser("WindowKeyEqualiser")); + } + + final OneInputStreamOperator windowOperator = + windowAggOperatorBuilder.build(); final OneInputTransformation transform = ExecNodeUtil.createOneInputTransformation( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala index 7918ac7cda7dc..e281000f4577e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala @@ -17,18 +17,22 @@ */ package org.apache.flink.table.planner.plan.utils +import org.apache.flink.configuration.ReadableConfig import org.apache.flink.table.api.{DataTypes, TableConfig, TableException, ValidationException} +import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.JBigDecimal import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexTableArgCall} import org.apache.flink.table.planner.functions.sql.{FlinkSqlOperatorTable, SqlWindowTableFunction} import org.apache.flink.table.planner.plan.`trait`.RelWindowProperties import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery -import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalAggregate, FlinkLogicalMatch, FlinkLogicalOverAggregate, FlinkLogicalRank, FlinkLogicalTableFunctionScan} +import org.apache.flink.table.planner.plan.nodes.logical._ import org.apache.flink.table.planner.plan.utils.AggregateUtil.inferAggAccumulatorNames import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED} import org.apache.flink.table.planner.typeutils.RowTypeUtils import org.apache.flink.table.runtime.groupwindow._ +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.types.logical.TimestampType import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.canBeTimeAttributeType @@ -343,6 +347,21 @@ object WindowUtil { } } + def isAsyncStateEnabled( + config: ReadableConfig, + windowAssigner: WindowAssigner, + aggInfoList: AggregateInfoList): Boolean = { + if (!config.get(ExecutionConfigOptions.TABLE_EXEC_ASYNC_STATE_ENABLED)) { + return false + } + // currently, unsliced assigner does not support async state + if (!windowAssigner.isInstanceOf[SliceAssigner]) { + return false + } + + AggregateUtil.isAsyncStateEnabled(config, aggInfoList) + } + // ------------------------------------------------------------------------------------------ // Private Helpers // ------------------------------------------------------------------------------------------ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala index 0986d390a22df..8485d1f418e4d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala @@ -21,7 +21,7 @@ import org.apache.flink.configuration.{Configuration, RestartStrategyOptions} import org.apache.flink.core.execution.CheckpointingMode import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ -import org.apache.flink.table.api.config.{AggregatePhaseStrategy, OptimizerConfigOptions} +import org.apache.flink.table.api.config.{AggregatePhaseStrategy, ExecutionConfigOptions, OptimizerConfigOptions} import org.apache.flink.table.api.config.AggregatePhaseStrategy._ import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.factories.TestValuesTableFactory.changelogRow @@ -46,7 +46,8 @@ import scala.collection.JavaConversions._ class WindowAggregateITCase( aggPhase: AggregatePhaseStrategy, state: StateBackendMode, - useTimestampLtz: Boolean) + useTimestampLtz: Boolean, + enableAsyncState: Boolean) extends StreamingWithStateTestBase(state) { // ------------------------------------------------------------------------------- @@ -146,6 +147,9 @@ class WindowAggregateITCase( configuration.set( RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, Duration.ofMillis(0)) + configuration.set( + ExecutionConfigOptions.TABLE_EXEC_ASYNC_STATE_ENABLED, + Boolean.box(enableAsyncState)) env.configure(configuration, Thread.currentThread.getContextClassLoader) FailingCollectionSource.reset() @@ -212,27 +216,6 @@ class WindowAggregateITCase( @TestTemplate def testEventTimeTumbleWindow(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-10T00:00,2020-10-10T00:00:05,4,11.10,5.0,1.0,2,Hi|Comment#1", "a,2020-10-10T00:00:05,2020-10-10T00:00:10,1,3.33,null,3.0,1,Comment#2", @@ -241,40 +224,21 @@ class WindowAggregateITCase( "b,2020-10-10T00:00:30,2020-10-10T00:00:35,1,3.33,3.0,3.0,1,Comment#3", "null,2020-10-10T00:00:30,2020-10-10T00:00:35,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg("TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", expected) } @TestTemplate def testEventTimeTumbleWindowWithOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '1' DAY, INTERVAL '8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T08:00,2020-10-10T08:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "b,2020-10-09T08:00,2020-10-10T08:00,4,14.43,6.0,3.0,3,Hello|Hi|Comment#3", "null,2020-10-09T08:00,2020-10-10T08:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + "TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '1' DAY, INTERVAL '8' HOUR)", + expected) } @TestTemplate @@ -311,118 +275,39 @@ class WindowAggregateITCase( @TestTemplate def testEventTimeTumbleWindowWithNegativeOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '1' DAY, INTERVAL '-8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T16:00,2020-10-10T16:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "b,2020-10-09T16:00,2020-10-10T16:00,4,14.43,6.0,3.0,3,Hello|Hi|Comment#3", "null,2020-10-09T16:00,2020-10-10T16:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + "TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '1' DAY, INTERVAL '-8' HOUR)", + expected) } @TestTemplate def testEventTimeTumbleWindow_GroupingSets(): Unit = { - val sql = - """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY GROUPING SETS((`name`),()), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(TumbleWindowGroupSetExpectedData.sorted.mkString("\n")) + verifyWindowAggWithGroupingSets( + "TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + "GROUPING SETS((`name`),())", + TumbleWindowGroupSetExpectedData) } @TestTemplate def testEventTimeTumbleWindow_Cube(): Unit = { - val sql = - """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY CUBE(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(TumbleWindowCubeExpectedData.sorted.mkString("\n")) + verifyWindowAggWithGroupingSets( + "TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + "CUBE(`name`)", + TumbleWindowCubeExpectedData) } @TestTemplate def testEventTimeTumbleWindow_Rollup(): Unit = { - val sql = - """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY ROLLUP(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(TumbleWindowRollupExpectedData.sorted.mkString("\n")) + verifyWindowAggWithGroupingSets( + "TUMBLE(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + "ROLLUP(`name`)", + TumbleWindowRollupExpectedData) } @TestTemplate @@ -529,28 +414,7 @@ class WindowAggregateITCase( } @TestTemplate - def testEventTimeHopWindow(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - + def testEventTimeHopWindowWithDistinct(): Unit = { val expected = Seq( "a,2020-10-09T23:59:55,2020-10-10T00:00:05,4,11.10,5.0,1.0,2,Hi|Comment#1", "a,2020-10-10T00:00,2020-10-10T00:00:10,6,19.98,5.0,1.0,3,Comment#2|Hi|Comment#1", @@ -564,38 +428,14 @@ class WindowAggregateITCase( "null,2020-10-10T00:00:25,2020-10-10T00:00:35,1,7.77,7.0,7.0,0,null", "null,2020-10-10T00:00:30,2020-10-10T00:00:40,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + "HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + expected) } @TestTemplate def testEventTimeHopWindowWithOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '12' HOUR, - | INTERVAL '1' DAY, - | INTERVAL '8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T08:00,2020-10-10T08:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "a,2020-10-09T20:00,2020-10-10T20:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", @@ -604,38 +444,22 @@ class WindowAggregateITCase( "null,2020-10-09T08:00,2020-10-10T08:00,1,7.77,7.0,7.0,0,null", "null,2020-10-09T20:00,2020-10-10T20:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + """ + |HOP( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '12' HOUR, + | INTERVAL '1' DAY, + | INTERVAL '8' HOUR) + |""".stripMargin, + expected + ) } @TestTemplate def testEventTimeHopWindowWithNegativeOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '12' HOUR, - | INTERVAL '1' DAY, - | INTERVAL '-8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T04:00,2020-10-10T04:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "a,2020-10-09T16:00,2020-10-10T16:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", @@ -644,121 +468,46 @@ class WindowAggregateITCase( "null,2020-10-09T04:00,2020-10-10T04:00,1,7.77,7.0,7.0,0,null", "null,2020-10-09T16:00,2020-10-10T16:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) - } - @TestTemplate - def testEventTimeHopWindow_GroupingSets(): Unit = { - val sql = + verifyWindowAgg( """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)) - |GROUP BY GROUPING SETS((`name`),()), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(HopWindowGroupSetExpectedData.sorted.mkString("\n")) - } - - @TestTemplate - def testEventTimeHopWindow_Cube(): Unit = { - val sql = - """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)) - |GROUP BY CUBE(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(HopWindowCubeExpectedData.sorted.mkString("\n")) + |HOP( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '12' HOUR, + | INTERVAL '1' DAY, + | INTERVAL '-8' HOUR) + |""".stripMargin, + expected + ) } @TestTemplate - def testEventTimeHopWindow_Rollup(): Unit = { - val sql = - """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)) - |GROUP BY ROLLUP(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(HopWindowRollupExpectedData.sorted.mkString("\n")) + def testEventTimeHopWindow_GroupingSets(): Unit = { + verifyWindowAggWithGroupingSets( + "HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + "GROUPING SETS((`name`),())", + HopWindowGroupSetExpectedData) } @TestTemplate - def testEventTimeCumulateWindow(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '5' SECOND, - | INTERVAL '15' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() + def testEventTimeHopWindow_Cube(): Unit = { + verifyWindowAggWithGroupingSets( + "HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + "CUBE(`name`)", + HopWindowCubeExpectedData) + } + + @TestTemplate + def testEventTimeHopWindow_Rollup(): Unit = { + verifyWindowAggWithGroupingSets( + "HOP(TABLE T1, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + "ROLLUP(`name`)", + HopWindowRollupExpectedData) + } + @TestTemplate + def testEventTimeCumulateWindow(): Unit = { val expected = Seq( "a,2020-10-10T00:00,2020-10-10T00:00:05,4,11.10,5.0,1.0,2,Hi|Comment#1", "a,2020-10-10T00:00,2020-10-10T00:00:10,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", @@ -775,75 +524,41 @@ class WindowAggregateITCase( "null,2020-10-10T00:00:30,2020-10-10T00:00:40,1,7.77,7.0,7.0,0,null", "null,2020-10-10T00:00:30,2020-10-10T00:00:45,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + """ + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '5' SECOND, + | INTERVAL '15' SECOND) + |""".stripMargin, + expected + ) } @TestTemplate def testEventTimeCumulateWindowWithOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '12' HOUR, - | INTERVAL '1' DAY, - | INTERVAL '8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T08:00,2020-10-10T08:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "b,2020-10-09T08:00,2020-10-10T08:00,4,14.43,6.0,3.0,3,Hello|Hi|Comment#3", "null,2020-10-09T08:00,2020-10-10T08:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + """ + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '12' HOUR, + | INTERVAL '1' DAY, + | INTERVAL '8' HOUR) + |""".stripMargin, + expected + ) } @TestTemplate def testEventTimeCumulateWindowWithNegativeOffset(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '12' HOUR, - | INTERVAL '1' DAY, - | INTERVAL '-8' HOUR)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T16:00,2020-10-10T04:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "a,2020-10-09T16:00,2020-10-10T16:00,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", @@ -852,104 +567,62 @@ class WindowAggregateITCase( "null,2020-10-09T16:00,2020-10-10T04:00,1,7.77,7.0,7.0,0,null", "null,2020-10-09T16:00,2020-10-10T16:00,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + """ + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '12' HOUR, + | INTERVAL '1' DAY, + | INTERVAL '-8' HOUR) + |""".stripMargin, + expected + ) } @TestTemplate def testEventTimeCumulateWindow_GroupingSets(): Unit = { - val sql = + verifyWindowAggWithGroupingSets( """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '5' SECOND, - | INTERVAL '15' SECOND)) - |GROUP BY GROUPING SETS((`name`),()), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(CumulateWindowGroupSetExpectedData.sorted.mkString("\n")) + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '5' SECOND, + | INTERVAL '15' SECOND) + |""".stripMargin, + "GROUPING SETS((`name`),())", + CumulateWindowGroupSetExpectedData + ) } @TestTemplate def testEventTimeCumulateWindow_Cube(): Unit = { - val sql = + verifyWindowAggWithGroupingSets( """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '5' SECOND, - | INTERVAL '15' SECOND)) - |GROUP BY Cube(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(CumulateWindowCubeExpectedData.sorted.mkString("\n")) + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '5' SECOND, + | INTERVAL '15' SECOND) + |""".stripMargin, + "Cube(`name`)", + CumulateWindowCubeExpectedData + ) } @TestTemplate def testEventTimeCumulateWindow_Rollup(): Unit = { - val sql = + verifyWindowAggWithGroupingSets( """ - |SELECT - | GROUPING_ID(`name`), - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | CUMULATE( - | TABLE T1, - | DESCRIPTOR(rowtime), - | INTERVAL '5' SECOND, - | INTERVAL '15' SECOND)) - |GROUP BY ROLLUP(`name`), window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(CumulateWindowRollupExpectedData.sorted.mkString("\n")) + |CUMULATE( + | TABLE T1, + | DESCRIPTOR(rowtime), + | INTERVAL '5' SECOND, + | INTERVAL '15' SECOND) + |""".stripMargin, + "ROLLUP(`name`)", + CumulateWindowRollupExpectedData + ) } @TestTemplate @@ -1035,58 +708,21 @@ class WindowAggregateITCase( @TestTemplate def testEventTimeTumbleWindowWithCDCSource(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`) - |FROM TABLE( - | TUMBLE(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-10T00:00,2020-10-10T00:00:05,3,29.99,22.0,2.0,2", "a,2020-10-10T00:00:05,2020-10-10T00:00:10,1,3.33,null,3.0,1", "b,2020-10-10T00:00:05,2020-10-10T00:00:10,2,6.66,6.0,3.0,2", "b,2020-10-10T00:00:15,2020-10-10T00:00:20,1,4.44,4.0,4.0,1" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + "TUMBLE(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + expected, + isCdcSource = true + ) } @TestTemplate def testEventTimeHopWindowWithCDCSource(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`) - |FROM TABLE( - | HOP(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-09T23:59:55,2020-10-10T00:00:05,3,29.99,22.0,2.0,2", "a,2020-10-10T00:00,2020-10-10T00:00:10,5,38.87,22.0,2.0,4", @@ -1096,32 +732,15 @@ class WindowAggregateITCase( "b,2020-10-10T00:00:10,2020-10-10T00:00:20,1,4.44,4.0,4.0,1", "b,2020-10-10T00:00:15,2020-10-10T00:00:25,1,4.44,4.0,4.0,1" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + "HOP(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '10' SECOND)", + expected, + isCdcSource = true + ) } @TestTemplate def testEventTimeCumulateWindowWithCDCSource(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`) - |FROM TABLE( - | CUMULATE(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-10T00:00,2020-10-10T00:00:05,3,29.99,22.0,2.0,2", "a,2020-10-10T00:00,2020-10-10T00:00:10,5,38.87,22.0,2.0,4", @@ -1132,8 +751,11 @@ class WindowAggregateITCase( "b,2020-10-10T00:00:15,2020-10-10T00:00:25,1,4.44,4.0,4.0,1", "b,2020-10-10T00:00:15,2020-10-10T00:00:30,1,4.44,4.0,4.0,1" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + "CUMULATE(TABLE T1_CDC, DESCRIPTOR(rowtime), INTERVAL '5' SECOND, INTERVAL '15' SECOND)", + expected, + isCdcSource = true + ) } @TestTemplate @@ -1190,27 +812,6 @@ class WindowAggregateITCase( @TestTemplate def testEventTimeSessionWindow(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`), - | concat_distinct_agg(`string`) - |FROM TABLE( - | SESSION(TABLE T1 PARTITION BY `name`, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-10T00:00:01,2020-10-10T00:00:13,6,19.98,5.0,1.0,3,Hi|Comment#1|Comment#2", "b,2020-10-10T00:00:06,2020-10-10T00:00:12,2,6.66,6.0,3.0,2,Hello|Hi", @@ -1218,8 +819,11 @@ class WindowAggregateITCase( "b,2020-10-10T00:00:34,2020-10-10T00:00:39,1,3.33,3.0,3.0,1,Comment#3", "null,2020-10-10T00:00:32,2020-10-10T00:00:37,1,7.77,7.0,7.0,0,null" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + + verifyWindowAgg( + "SESSION(TABLE T1 PARTITION BY `name`, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + expected + ) } @TestTemplate @@ -1261,33 +865,16 @@ class WindowAggregateITCase( @TestTemplate def testEventTimeSessionWindowWithCDCSource(): Unit = { - val sql = - """ - |SELECT - | `name`, - | window_start, - | window_end, - | COUNT(*), - | SUM(`bigdec`), - | MAX(`double`), - | MIN(`float`), - | COUNT(DISTINCT `string`) - |FROM TABLE( - | SESSION(TABLE T1_CDC PARTITION BY `name`, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)) - |GROUP BY `name`, window_start, window_end - """.stripMargin - - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toDataStream.addSink(sink) - env.execute() - val expected = Seq( "a,2020-10-10T00:00:01,2020-10-10T00:00:13,5,38.87,22.0,2.0,4", "b,2020-10-10T00:00:06,2020-10-10T00:00:12,2,6.66,6.0,3.0,2", "b,2020-10-10T00:00:16,2020-10-10T00:00:21,1,4.44,4.0,4.0,1" ) - assertThat(sink.getAppendResults.sorted.mkString("\n")) - .isEqualTo(expected.sorted.mkString("\n")) + verifyWindowAgg( + "SESSION(TABLE T1_CDC PARTITION BY `name`, DESCRIPTOR(rowtime), INTERVAL '5' SECOND)", + expected, + isCdcSource = true + ) } @TestTemplate @@ -1395,18 +982,124 @@ class WindowAggregateITCase( } } } + + private def verifyWindowAgg( + tvfFromClause: String, + allExpectedData: Seq[String], + isCdcSource: Boolean = false): Unit = { + val aggFunctionsWithDataView = + if (isCdcSource) { + // concat_distinct_agg does not support retract + """ + |,COUNT(DISTINCT `string`) + |""".stripMargin + } else { + """ + |,COUNT(DISTINCT `string`) + |,concat_distinct_agg(`string`) + |""".stripMargin + } + + val sql = + s""" + |SELECT + | `name` + | ,window_start + | ,window_end + | ,COUNT(*) + | ,SUM(`bigdec`) + | ,MAX(`double`) + | ,MIN(`float`) + | -- agg function with data view does not support async state yet + | ${if (!enableAsyncState) aggFunctionsWithDataView else ""} + |FROM TABLE($tvfFromClause) + |GROUP BY `name`, window_start, window_end + |""".stripMargin + + // remove the last data used to verify 'COUNT(DISTINCT `string`)' + // and concat_distinct_agg(`string`) + val numToDropWithAsyncState = if (isCdcSource) 1 else 2 + executeAndVerify(sql, allExpectedData, numToDropWithAsyncState) + } + + private def verifyWindowAggWithGroupingSets( + tvfFromClause: String, + groupingSetClause: String, + allExpectedData: Seq[String]): Unit = { + val aggFunctionsWithDataView = + """ + |,COUNT(DISTINCT `string`) + |,concat_distinct_agg(`string`) + |""".stripMargin + + val sql = + s""" + |SELECT + | GROUPING_ID(`name`), + | `name` + | ,window_start + | ,window_end + | ,COUNT(*) + | ,SUM(`bigdec`) + | ,MAX(`double`) + | ,MIN(`float`) + | -- agg function with data view does not support async state yet + | ${if (!enableAsyncState) aggFunctionsWithDataView else ""} + |FROM TABLE($tvfFromClause) + |GROUP BY $groupingSetClause, window_start, window_end + |""".stripMargin + + // remove the last data used to verify 'COUNT(DISTINCT `string`)' + // and concat_distinct_agg(`string`) + executeAndVerify(sql, allExpectedData, 2) + } + + private def executeAndVerify( + query: String, + allExpectedData: Seq[String], + numToDropWithAsyncState: Int): Unit = { + val sink = new TestingAppendSink + tEnv.sqlQuery(query).toDataStream.addSink(sink) + env.execute() + + val expected = filterTailDataIfNecessary(allExpectedData, numToDropWithAsyncState) + assertThat(sink.getAppendResults.sorted.mkString("\n")) + .isEqualTo(expected.sorted.mkString("\n")) + } + + private def filterTailDataIfNecessary( + data: Seq[String], + numToDropWithAsyncState: Int): Seq[String] = { + if (!enableAsyncState) { + return data + } + data + .map( + line => { + val parts = line.split(",") + if (parts.length >= numToDropWithAsyncState) { + parts.dropRight(numToDropWithAsyncState) + } else { + Array.empty[String] + } + }) + .map(_.mkString(",")) + } } object WindowAggregateITCase { - @Parameters(name = "AggPhase={0}, StateBackend={1}, UseTimestampLtz = {2}") + @Parameters( + name = "AggPhase={0}, StateBackend={1}, UseTimestampLtz = {2}, EnableAsyncState = {3}") def parameters(): util.Collection[Array[java.lang.Object]] = { Seq[Array[AnyRef]]( // we do not test all cases to simplify the test matrix - Array(ONE_PHASE, HEAP_BACKEND, java.lang.Boolean.TRUE), - Array(TWO_PHASE, HEAP_BACKEND, java.lang.Boolean.FALSE), - Array(ONE_PHASE, ROCKSDB_BACKEND, java.lang.Boolean.FALSE), - Array(TWO_PHASE, ROCKSDB_BACKEND, java.lang.Boolean.TRUE) + Array(ONE_PHASE, HEAP_BACKEND, java.lang.Boolean.TRUE, Boolean.box(false)), + Array(ONE_PHASE, HEAP_BACKEND, java.lang.Boolean.TRUE, Boolean.box(true)), + Array(ONE_PHASE, HEAP_BACKEND, java.lang.Boolean.FALSE, Boolean.box(true)), + Array(TWO_PHASE, HEAP_BACKEND, java.lang.Boolean.FALSE, Boolean.box(false)), + Array(ONE_PHASE, ROCKSDB_BACKEND, java.lang.Boolean.FALSE, Boolean.box(false)), + Array(TWO_PHASE, ROCKSDB_BACKEND, java.lang.Boolean.TRUE, Boolean.box(false)) ) } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/AsyncStateTableStreamOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/AsyncStateTableStreamOperator.java index 9807352470f30..e74d1e5c969cf 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/AsyncStateTableStreamOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/AsyncStateTableStreamOperator.java @@ -18,7 +18,9 @@ package org.apache.flink.table.runtime.operators; +import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperator; +import org.apache.flink.runtime.execution.Environment; import org.apache.flink.streaming.api.TimerService; import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.streaming.api.watermark.Watermark; @@ -54,6 +56,20 @@ public boolean useSplittableTimers() { return true; } + /** Compute memory size from memory faction. */ + public long computeMemorySize() { + final Environment environment = getContainingTask().getEnvironment(); + return environment + .getMemoryManager() + .computeMemorySize( + getOperatorConfig() + .getManagedMemoryFractionOperatorUseCaseOfSlot( + ManagedMemoryUseCase.OPERATOR, + environment.getJobConfiguration(), + environment.getTaskManagerInfo().getConfiguration(), + environment.getUserCodeClassLoader().asClassLoader())); + } + @Override public Watermark preProcessWatermark(Watermark mark) throws Exception { currentWatermark = mark.getTimestamp(); @@ -94,25 +110,25 @@ public long currentWatermark() { @Override public void registerProcessingTimeTimer(long time) { throw new UnsupportedOperationException( - "Setting timers is only supported on a keyed streams."); + "Setting timers is only supported on keyed streams."); } @Override public void registerEventTimeTimer(long time) { throw new UnsupportedOperationException( - "Setting timers is only supported on a keyed streams."); + "Setting timers is only supported on keyed streams."); } @Override public void deleteProcessingTimeTimer(long time) { throw new UnsupportedOperationException( - "Delete timers is only supported on a keyed streams."); + "Delete timers is only supported on keyed streams."); } @Override public void deleteEventTimeTimer(long time) { throw new UnsupportedOperationException( - "Delete timers is only supported on a keyed streams."); + "Delete timers is only supported on keyed streams."); } public TimerService timerService() { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java index d3b1d910c86f8..6ac269e40738a 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableStreamOperator.java @@ -115,25 +115,25 @@ public long currentWatermark() { @Override public void registerProcessingTimeTimer(long time) { throw new UnsupportedOperationException( - "Setting timers is only supported on a keyed streams."); + "Setting timers is only supported on keyed streams."); } @Override public void registerEventTimeTimer(long time) { throw new UnsupportedOperationException( - "Setting timers is only supported on a keyed streams."); + "Setting timers is only supported on keyed streams."); } @Override public void deleteProcessingTimeTimer(long time) { throw new UnsupportedOperationException( - "Delete timers is only supported on a keyed streams."); + "Delete timers is only supported on keyed streams."); } @Override public void deleteEventTimeTimer(long time) { throw new UnsupportedOperationException( - "Delete timers is only supported on a keyed streams."); + "Delete timers is only supported on keyed streams."); } public TimerService timerService() { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateRecordsWindowBuffer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateRecordsWindowBuffer.java new file mode 100644 index 0000000000000..22348dfed687f --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateRecordsWindowBuffer.java @@ -0,0 +1,226 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.buffers; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.generated.RecordEqualiser; +import org.apache.flink.table.runtime.operators.window.async.tvf.combines.AsyncStateRecordsCombiner; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.AsyncStateKeyContext; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncState; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; +import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; +import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; +import org.apache.flink.table.runtime.typeutils.WindowKeySerializer; +import org.apache.flink.table.runtime.util.KeyValueIterator; +import org.apache.flink.table.runtime.util.WindowKey; +import org.apache.flink.table.runtime.util.collections.binary.BytesMap; +import org.apache.flink.table.runtime.util.collections.binary.WindowBytesMultiMap; + +import javax.annotation.Nullable; + +import java.io.EOFException; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import static org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + +/** + * An implementation of {@link AsyncStateWindowBuffer} that buffers input elements in a {@link + * WindowBytesMultiMap} and combines buffered elements into async state when flushing. + */ +public final class AsyncStateRecordsWindowBuffer implements AsyncStateWindowBuffer { + + private final AsyncStateRecordsCombiner combineFunction; + private final WindowBytesMultiMap recordsBuffer; + private final WindowKey reuseWindowKey; + private final AbstractRowDataSerializer recordSerializer; + private final ZoneId shiftTimeZone; + private final RecordEqualiser keyEqualiser; + private final AsyncStateKeyContext keyContext; + + private long minSliceEnd = Long.MAX_VALUE; + + public AsyncStateRecordsWindowBuffer( + Object operatorOwner, + MemoryManager memoryManager, + long memorySize, + AsyncStateRecordsCombiner combineFunction, + PagedTypeSerializer keySer, + AbstractRowDataSerializer inputSer, + RecordEqualiser keyEqualiser, + AsyncStateKeyContext keyContext, + ZoneId shiftTimeZone) { + this.combineFunction = combineFunction; + this.recordsBuffer = + new WindowBytesMultiMap( + operatorOwner, memoryManager, memorySize, keySer, inputSer.getArity()); + this.recordSerializer = inputSer; + this.keyEqualiser = keyEqualiser; + this.keyContext = keyContext; + this.reuseWindowKey = new WindowKeySerializer(keySer).createInstance(); + this.shiftTimeZone = shiftTimeZone; + } + + @Override + public StateFuture addElement(RowData dataKey, long sliceEnd, RowData element) + throws Exception { + StateFuture resultFuture = REUSABLE_VOID_STATE_FUTURE; + + // track the lowest trigger time, if watermark exceeds the trigger time, + // it means there are some elements in the buffer belong to a window going to be fired, + // and we need to flush the buffer into state for firing. + minSliceEnd = Math.min(sliceEnd, minSliceEnd); + + reuseWindowKey.replace(sliceEnd, dataKey); + BytesMap.LookupInfo> lookup = + recordsBuffer.lookup(reuseWindowKey); + try { + recordsBuffer.append(lookup, recordSerializer.toBinaryRow(element)); + } catch (EOFException e) { + // buffer is full, flush it to state + resultFuture = flush(dataKey); + // remember to add the input element again + addElement(dataKey, sliceEnd, element); + } + return resultFuture; + } + + @Override + public StateFuture advanceProgress(@Nullable RowData currentKey, long progress) + throws Exception { + if (isWindowFired(minSliceEnd, progress, shiftTimeZone)) { + // there should be some window to be fired, flush buffer to state first + return flush(currentKey); + } + return REUSABLE_VOID_STATE_FUTURE; + } + + @Override + public StateFuture flush(@Nullable RowData currentKey) throws Exception { + StateFuture flushFuture = REUSABLE_VOID_STATE_FUTURE; + if (recordsBuffer.getNumKeys() > 0) { + // due to the delayed processing of async requests, all objects cannot be reused, so + // they must be copied. + KeyValueIterator> entryIterator = + recordsBuffer.getEntryIterator(true); + while (entryIterator.advanceNext()) { + WindowKey windowKey = entryIterator.getKey(); + long window = windowKey.getWindow(); + List allData = itertorToList(entryIterator.getValue()); + if (currentKey != null && keyEqualiser.equals(currentKey, windowKey.getKey())) { + flushFuture = combineFunction.asyncCombine(window, allData.iterator()); + } else { + // no need to wait for combining the records excluding current key + keyContext.asyncProcessWithKey( + windowKey.getKey(), + () -> combineFunction.asyncCombine(window, allData.iterator())); + } + } + recordsBuffer.reset(); + // reset trigger time + minSliceEnd = Long.MAX_VALUE; + } + return flushFuture; + } + + /** + * Convert iterator to list. + * + *

This may put some pressure on heap memory since the data in the iterator comes from + * managed memory. We can optimize this method once we come up with a better approach. + */ + private List itertorToList(Iterator records) { + List list = new ArrayList<>(); + while (records.hasNext()) { + list.add(records.next()); + } + return list; + } + + @Override + public void close() throws Exception { + recordsBuffer.free(); + combineFunction.close(); + } + + // ------------------------------------------------------------------------------------------ + // Factory + // ------------------------------------------------------------------------------------------ + + /** + * Factory to create {@link AsyncStateRecordsWindowBuffer} with {@link + * AsyncStateRecordsCombiner.Factory}. + */ + public static final class Factory implements AsyncStateWindowBuffer.Factory { + + private static final long serialVersionUID = 1L; + + private final PagedTypeSerializer keySer; + private final AbstractRowDataSerializer inputSer; + private final AsyncStateRecordsCombiner.Factory factory; + private final GeneratedRecordEqualiser generatedKeyEqualiser; + + public Factory( + PagedTypeSerializer keySer, + AbstractRowDataSerializer inputSer, + AsyncStateRecordsCombiner.Factory combinerFactory, + GeneratedRecordEqualiser generatedKeyEqualiser) { + this.keySer = keySer; + this.inputSer = inputSer; + this.factory = combinerFactory; + this.generatedKeyEqualiser = generatedKeyEqualiser; + } + + @Override + public AsyncStateWindowBuffer create( + Object operatorOwner, + MemoryManager memoryManager, + long memorySize, + RuntimeContext runtimeContext, + WindowTimerService timerService, + AsyncStateKeyContext keyContext, + WindowAsyncState windowState, + boolean isEventTime, + ZoneId shiftTimeZone) + throws Exception { + AsyncStateRecordsCombiner combiner = + factory.createRecordsCombiner( + runtimeContext, timerService, windowState, isEventTime); + RecordEqualiser keyEqualiser = + generatedKeyEqualiser.newInstance(runtimeContext.getUserCodeClassLoader()); + return new AsyncStateRecordsWindowBuffer( + operatorOwner, + memoryManager, + memorySize, + combiner, + keySer, + inputSer, + keyEqualiser, + keyContext, + shiftTimeZone); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateWindowBuffer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateWindowBuffer.java new file mode 100644 index 0000000000000..dd7344f63ad0d --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/buffers/AsyncStateWindowBuffer.java @@ -0,0 +1,133 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.buffers; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.AsyncStateKeyContext; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncState; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.time.ZoneId; + +/** + * A buffer that buffers data in memory and flushes many values to async state together at a time to + * avoid frequently accessing async state, or flushes to output to reduce shuffling data. + */ +public interface AsyncStateWindowBuffer { + + /** + * Adds an element with associated key into the buffer. The buffer may temporarily buffer the + * element, or immediately write it to the stream. + * + *

It may be that adding this element fills up an internal buffer and causes the buffer + * flushing of a batch of internally buffered elements. + * + * @param dataKey the key associated with the element + * @param element The element to add. + * @throws Exception Thrown, if the element cannot be added to the buffer, or if the flushing + * throws an exception. + */ + StateFuture addElement(RowData dataKey, long window, RowData element) throws Exception; + + /** + * Advances the progress time, the progress time is watermark if working in event-time mode, or + * current processing time if working in processing-time mode. + * + *

This will potentially flush buffered data into states or to the output stream, because the + * watermark advancement may be in a very small step, but we don't need to flush buffered data + * for every watermark advancement. + * + *

Note: There may be multiple different keys within the buffer. When flushing them to the + * async state, only the async state request for the current key of the operator will be + * returned as a {@link StateFuture}. Requests for async states for other keys will not be + * waited on. + * + * @param currentKey the current key when processing and is used to return the result of + * accessing async state associated with the same key. If it is null, it means that the + * returns of asynchronous state requests for all keys will not be awaited. + * @param progress the current progress time + * @return the future of the flush operation about current key if the current key is not null, + * else a {@link StateFutureUtils#completedVoidFuture()} will be returned. + */ + StateFuture advanceProgress(@Nullable RowData currentKey, long progress) throws Exception; + + /** + * Flushes all intermediate buffered data to the underlying backend or output stream. + * + *

Note: There may be multiple different keys within the buffer. When flushing them to the + * async state, only the async state request for the current key of the operator will be + * returned as a {@link StateFuture}. Requests for async states for other keys will not be + * waited on. + * + * @param currentKey the current key when processing and is used to return the result of + * accessing async state associated with the same key. If it is null, it means that the + * returns of asynchronous state requests for all keys will not be awaited. + * @return the future of the flush operation about current key if the current key is not null, + * else a {@link StateFutureUtils#completedVoidFuture()} will be returned. + * @throws Exception Thrown if the buffer cannot be flushed, or if the output stream throws an + * exception. + */ + StateFuture flush(@Nullable RowData currentKey) throws Exception; + + /** Release resources allocated by this buffer. */ + void close() throws Exception; + + // ------------------------------------------------------------------------ + + /** A factory that creates a {@link WindowBuffer} with async state. */ + @FunctionalInterface + interface Factory extends Serializable { + + /** + * Creates a {@link WindowBuffer} that buffers elements in memory before flushing. + * + * @param operatorOwner the owner of the operator + * @param memoryManager the manager that governs memory by Flink framework + * @param memorySize the managed memory size can be used by this operator + * @param runtimeContext the current {@link RuntimeContext} + * @param timerService the service to register event-time and processing-time timers + * @param keyContext the state context to accessing states + * @param windowState the window async state to flush buffered data into. + * @param isEventTime indicates whether the operator works in event-time or processing-time + * mode, used for register corresponding timers. + * @param shiftTimeZone the shift timezone of the window + * @throws IOException thrown if the buffer can't be opened + */ + AsyncStateWindowBuffer create( + Object operatorOwner, + MemoryManager memoryManager, + long memorySize, + RuntimeContext runtimeContext, + WindowTimerService timerService, + AsyncStateKeyContext keyContext, + WindowAsyncState windowState, + boolean isEventTime, + ZoneId shiftTimeZone) + throws Exception; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/combines/AsyncStateAggCombiner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/combines/AsyncStateAggCombiner.java new file mode 100644 index 0000000000000..d2ff222f285d1 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/combines/AsyncStateAggCombiner.java @@ -0,0 +1,147 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.combines; + +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.dataview.UnsupportedStateDataViewStore; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.window.async.tvf.combines.AsyncStateRecordsCombiner; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncState; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncValueState; +import org.apache.flink.table.runtime.operators.window.tvf.combines.RecordsCombiner; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; + +import java.time.ZoneId; +import java.util.Iterator; + +import static org.apache.flink.table.data.util.RowDataUtil.isAccumulateMsg; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + +/** + * An implementation of {@link RecordsCombiner} that accumulates input records into the window + * accumulator with async state. + */ +public class AsyncStateAggCombiner implements AsyncStateRecordsCombiner { + + /** The service to register event-time or processing-time timers. */ + private final WindowTimerService timerService; + + /** The state stores window accumulators. */ + private final WindowAsyncValueState accState; + + /** Function used to handle all aggregates. */ + private final NamespaceAggsHandleFunction aggregator; + + /** Whether the operator works in event-time mode, used to indicate registering which timer. */ + private final boolean isEventTime; + + public AsyncStateAggCombiner( + WindowTimerService timerService, + WindowAsyncValueState accState, + NamespaceAggsHandleFunction aggregator, + boolean isEventTime) { + this.timerService = timerService; + this.accState = accState; + this.aggregator = aggregator; + this.isEventTime = isEventTime; + } + + @Override + public StateFuture asyncCombine(Long window, Iterator records) throws Exception { + StateFuture resultFuture = + // step 1: get the accumulator for the current key and window + accState.asyncValue(window) + .thenCompose( + acc -> { + if (acc == null) { + acc = aggregator.createAccumulators(); + } + + // step 2: set accumulator to function + aggregator.setAccumulators(window, acc); + + // step 3: do accumulate + while (records.hasNext()) { + RowData record = records.next(); + if (isAccumulateMsg(record)) { + aggregator.accumulate(record); + } else { + aggregator.retract(record); + } + } + + // step 4: update accumulator into state + acc = aggregator.getAccumulators(); + + return accState.asyncUpdate(window, acc); + }); + + // step 5: register timer for current window + if (isEventTime) { + long currentWatermark = timerService.currentWatermark(); + ZoneId shiftTimeZone = timerService.getShiftTimeZone(); + // the registered window timer shouldn't been triggered + if (!isWindowFired(window, currentWatermark, shiftTimeZone)) { + timerService.registerEventTimeWindowTimer(window); + } + } + // we don't need register processing-time timer, because we already register them + // per-record in AbstractWindowAggProcessor.processElement() + return resultFuture; + } + + @Override + public void close() throws Exception { + aggregator.close(); + } + + // ---------------------------------------------------------------------------------------- + // Factory + // ---------------------------------------------------------------------------------------- + + /** Factory to create {@link AsyncStateAggCombiner}. */ + public static final class Factory implements AsyncStateRecordsCombiner.Factory { + private static final long serialVersionUID = 1L; + + private final GeneratedNamespaceAggsHandleFunction genAggsHandler; + + public Factory(GeneratedNamespaceAggsHandleFunction genAggsHandler) { + this.genAggsHandler = genAggsHandler; + } + + @Override + public AsyncStateRecordsCombiner createRecordsCombiner( + RuntimeContext runtimeContext, + WindowTimerService timerService, + WindowAsyncState windowState, + boolean isEventTime) + throws Exception { + final NamespaceAggsHandleFunction aggregator = + genAggsHandler.newInstance(runtimeContext.getUserCodeClassLoader()); + aggregator.open(new UnsupportedStateDataViewStore(runtimeContext)); + WindowAsyncValueState windowValueState = + (WindowAsyncValueState) windowState; + return new AsyncStateAggCombiner( + timerService, windowValueState, aggregator, isEventTime); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateSliceWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateSliceWindowAggProcessor.java new file mode 100644 index 0000000000000..e3bbc70e4d6ad --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateSliceWindowAggProcessor.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.operators.aggregate.asyncwindow.processors; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer; +import org.apache.flink.table.runtime.operators.window.MergeCallback; +import org.apache.flink.table.runtime.operators.window.async.tvf.slicing.AsyncStateSlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl; + +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_TRUE_STATE_FUTURE; +import static org.apache.flink.table.runtime.util.AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.getNextTriggerWatermark; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + +/** A base implementation of {@link AsyncStateSlicingWindowProcessor} for window aggregate. */ +public abstract class AbstractAsyncStateSliceWindowAggProcessor + extends AbstractAsyncStateWindowAggProcessor + implements AsyncStateSlicingWindowProcessor { + + protected final AsyncStateWindowBuffer.Factory windowBufferFactory; + protected final SliceAssigner sliceAssigner; + protected final long windowInterval; + + // ---------------------------------------------------------------------------------------- + + /** The next progress to trigger windows. */ + private transient long nextTriggerProgress; + + protected transient AsyncStateWindowBuffer windowBuffer; + + public AbstractAsyncStateSliceWindowAggProcessor( + GeneratedNamespaceAggsHandleFunction genAggsHandler, + AsyncStateWindowBuffer.Factory bufferFactory, + SliceAssigner sliceAssigner, + TypeSerializer accSerializer, + int indexOfCountStar, + ZoneId shiftTimeZone) { + super( + genAggsHandler, + sliceAssigner, + accSerializer, + sliceAssigner.isEventTime(), + indexOfCountStar, + shiftTimeZone, + Long.MIN_VALUE); + this.windowBufferFactory = bufferFactory; + this.sliceAssigner = sliceAssigner; + this.windowInterval = sliceAssigner.getSliceEndInterval(); + } + + @Override + public void open(AsyncStateContext context) throws Exception { + super.open(context); + this.windowBuffer = + windowBufferFactory.create( + ctx.getOperatorOwner(), + ctx.getMemoryManager(), + ctx.getMemorySize(), + ctx.getRuntimeContext(), + windowTimerService, + ctx.getAsyncKeyContext(), + windowState, + isEventTime, + shiftTimeZone); + + this.nextTriggerProgress = Long.MIN_VALUE; + } + + @Override + protected WindowTimerService getWindowTimerService() { + return new SlicingWindowTimerServiceImpl(ctx.getTimerService(), shiftTimeZone); + } + + @Override + public StateFuture processElement(RowData key, RowData element) throws Exception { + long sliceEnd = sliceAssigner.assignSliceEnd(element, clockService); + if (!isEventTime) { + // always register processing time for every element when processing time mode + windowTimerService.registerProcessingTimeWindowTimer(sliceEnd); + } + + if (isEventTime && isWindowFired(sliceEnd, currentProgress, shiftTimeZone)) { + // the assigned slice has been triggered, which means current element is late, + // but maybe not need to drop + long lastWindowEnd = sliceAssigner.getLastWindowEnd(sliceEnd); + if (isWindowFired(lastWindowEnd, currentProgress, shiftTimeZone)) { + // the last window has been triggered, so the element can be dropped now + return REUSABLE_TRUE_STATE_FUTURE; + } else { + StateFuture addElementFuture = + sliceStateMergeTarget(sliceEnd) + .thenCompose( + targetStateWindow -> + windowBuffer.addElement( + key, targetStateWindow, element)); + // we need to register a timer for the next unfired window, + // because this may the first time we see elements under the key + long unfiredFirstWindow = sliceEnd; + while (isWindowFired(unfiredFirstWindow, currentProgress, shiftTimeZone)) { + unfiredFirstWindow += windowInterval; + } + windowTimerService.registerEventTimeWindowTimer(unfiredFirstWindow); + return addElementFuture.thenApply(VOID -> false); + } + } else { + // the assigned slice hasn't been triggered, accumulate into the assigned slice + return windowBuffer.addElement(key, sliceEnd, element).thenApply(VOID -> false); + } + } + + /** + * Returns the slice state target to merge the given slice into when firing windows. For + * unshared windows, there should no merging happens, so the merge target should be just the + * given {@code sliceToMerge}. For shared windows, the merge target should be the shared slice + * state. + * + * @see SliceSharedAssigner#mergeSlices(long, MergeCallback) + */ + protected abstract StateFuture sliceStateMergeTarget(long sliceToMerge) throws Exception; + + @Override + public StateFuture advanceProgress(RowData currentKey, long progress) throws Exception { + StateFuture advanceFuture = REUSABLE_VOID_STATE_FUTURE; + if (progress > currentProgress) { + currentProgress = progress; + if (currentProgress >= nextTriggerProgress) { + // in order to buffer as much as possible data, we only need to call + // advanceProgress() when currentWatermark may trigger window. + // this is a good optimization when receiving late but un-dropped events, because + // they will register small timers and normal watermark will flush the buffer + advanceFuture = windowBuffer.advanceProgress(currentKey, currentProgress); + nextTriggerProgress = + getNextTriggerWatermark( + currentProgress, windowInterval, shiftTimeZone, useDayLightSaving); + } + } + return advanceFuture; + } + + @Override + public StateFuture prepareCheckpoint() throws Exception { + return windowBuffer.flush(null); + } + + @Override + public StateFuture clearWindow(long timerTimestamp, Long windowEnd) throws Exception { + Iterable expires = sliceAssigner.expiredSlices(windowEnd); + List> allClearFutures = new ArrayList<>(); + for (Long slice : expires) { + allClearFutures.add(windowState.asyncClear(slice)); + aggregator.cleanup(slice); + } + return StateFutureUtils.combineAll(allClearFutures).thenAccept(VOID -> {}); + } + + @Override + public void close() throws Exception { + super.close(); + if (windowBuffer != null) { + windowBuffer.close(); + } + } + + @Override + public TypeSerializer createWindowSerializer() { + return LongSerializer.INSTANCE; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateWindowAggProcessor.java new file mode 100644 index 0000000000000..a5f9982275e43 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AbstractAsyncStateWindowAggProcessor.java @@ -0,0 +1,121 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors; + +import org.apache.flink.api.common.state.v2.ValueState; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.state.v2.ValueStateDescriptor; +import org.apache.flink.runtime.state.v2.internal.InternalValueState; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.dataview.UnsupportedStateDataViewStore; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.WindowAggProcessorBase; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowProcessor; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncValueState; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigners; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.time.ZoneId; +import java.util.function.Function; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** A base class for window aggregate processors with async state. */ +public abstract class AbstractAsyncStateWindowAggProcessor + extends WindowAggProcessorBase> + implements AsyncStateWindowProcessor { + + private static final long serialVersionUID = 1L; + + protected final WindowIsEmptyChecker emptyChecker; + + private final W defaultWindow; + + // ---------------------------------------------------------------------------------------- + + /** state schema: [key, window, accumulator]. */ + protected transient WindowAsyncValueState windowState; + + public AbstractAsyncStateWindowAggProcessor( + GeneratedNamespaceAggsHandleFunction genAggsHandler, + WindowAssigner sliceAssigner, + TypeSerializer accSerializer, + boolean isEventTime, + int indexOfCountStar, + ZoneId shiftTimeZone, + W defaultWindow) { + super(genAggsHandler, accSerializer, isEventTime, shiftTimeZone); + this.emptyChecker = new WindowIsEmptyChecker(indexOfCountStar, sliceAssigner); + this.defaultWindow = defaultWindow; + } + + @Override + public void open(AsyncStateContext context) throws Exception { + super.open(context); + + ValueState state = + ctx.getAsyncKeyContext() + .getAsyncKeyedStateBackend() + .getOrCreateKeyedState( + defaultWindow, + createWindowSerializer(), + new ValueStateDescriptor<>("window-aggs", accSerializer)); + this.windowState = + new WindowAsyncValueState<>((InternalValueState) state); + } + + @Override + protected final void prepareAggregator() throws Exception { + this.aggregator = + genAggsHandler.newInstance(ctx.getRuntimeContext().getUserCodeClassLoader()); + this.aggregator.open(new UnsupportedStateDataViewStore(ctx.getRuntimeContext())); + } + + /** A checker that checks whether the window is empty. */ + protected static final class WindowIsEmptyChecker + implements Function, Serializable { + private static final long serialVersionUID = 1L; + + private final int indexOfCountStar; + + private WindowIsEmptyChecker(int indexOfCountStar, WindowAssigner assigner) { + if (assigner instanceof SliceAssigners.HoppingSliceAssigner) { + checkArgument( + indexOfCountStar >= 0, + "Hopping window requires a COUNT(*) in the aggregate functions."); + } + this.indexOfCountStar = indexOfCountStar; + } + + @Override + public Boolean apply(@Nullable RowData acc) { + if (indexOfCountStar < 0) { + return false; + } + try { + return acc == null || acc.getLong(indexOfCountStar) == 0; + } catch (Exception e) { + throw new RuntimeException(e.getMessage(), e); + } + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java new file mode 100644 index 0000000000000..cf79cc160724b --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceSharedWindowAggProcessor.java @@ -0,0 +1,207 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer; +import org.apache.flink.table.runtime.operators.window.async.AsyncMergeCallback; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; + +/** + * A window aggregate processor implementation which works for {@link SliceSharedAssigner} with + * async state api, e.g. hopping windows and cumulative windows. + */ +public final class AsyncStateSliceSharedWindowAggProcessor + extends AbstractAsyncStateSliceWindowAggProcessor + implements AsyncMergeCallback> { + + private static final long serialVersionUID = 1L; + + private final SliceSharedAssigner sliceSharedAssigner; + + public AsyncStateSliceSharedWindowAggProcessor( + GeneratedNamespaceAggsHandleFunction genAggsHandler, + AsyncStateWindowBuffer.Factory bufferFactory, + SliceSharedAssigner sliceAssigner, + TypeSerializer accSerializer, + int indexOfCountStar, + ZoneId shiftTimeZone) { + super( + genAggsHandler, + bufferFactory, + sliceAssigner, + accSerializer, + indexOfCountStar, + shiftTimeZone); + this.sliceSharedAssigner = sliceAssigner; + } + + @Override + public StateFuture fireWindow(long timerTimestamp, Long windowEnd) throws Exception { + return sliceSharedAssigner + .asyncMergeSlices(windowEnd, this) + .thenAccept( + accAndAggResult -> { + if (!emptyChecker.apply(accAndAggResult.f0)) { + // if the triggered window is an empty window, we shouldn't emit it + collect( + ctx.getAsyncKeyContext().getCurrentKey(), + accAndAggResult.f1); + } + + // we should register next window timer here, + // because slices are shared, maybe no elements arrived for the next + // slices + Optional nextWindowEndOptional = + sliceSharedAssigner.nextTriggerWindow( + windowEnd, accAndAggResult.f0, emptyChecker); + if (nextWindowEndOptional.isPresent()) { + long nextWindowEnd = nextWindowEndOptional.get(); + if (sliceSharedAssigner.isEventTime()) { + windowTimerService.registerEventTimeWindowTimer(nextWindowEnd); + } else { + windowTimerService.registerProcessingTimeWindowTimer( + nextWindowEnd); + } + } + }); + } + + @Override + public StateFuture> asyncMerge( + @Nullable Long mergeResult, Iterable toBeMerged, Long resultNamespace) + throws Exception { + // get base accumulator + final StateFuture accOfMergeResultFuture; + if (mergeResult == null) { + // null means the merged is not on state, create a new acc + accOfMergeResultFuture = + StateFutureUtils.completedFuture(aggregator.createAccumulators()); + } else { + accOfMergeResultFuture = + windowState + .asyncValue(mergeResult) + .thenApply( + stateAcc -> { + if (stateAcc == null) { + return aggregator.createAccumulators(); + } else { + return stateAcc; + } + }); + } + + StateFuture>> allAccOfSlicesToBeMergedFuture = + collectAccOfSlicesToBeMerged(toBeMerged); + + return accOfMergeResultFuture + .thenCombine( + allAccOfSlicesToBeMergedFuture, + (accOfMergeResult, allAccOfSlicesToBeMerged) -> { + // set base accumulator + aggregator.setAccumulators(mergeResult, accOfMergeResult); + + // merge slice accumulators + for (Tuple2 sliceAndAcc : allAccOfSlicesToBeMerged) { + RowData sliceAcc = sliceAndAcc.f1; + if (sliceAcc != null) { + aggregator.merge(sliceAndAcc.f0, sliceAcc); + } + } + + return Tuple2.of( + aggregator.getAccumulators(), + aggregator.getValue(resultNamespace)); + }) + .thenCompose( + accAndAggResult -> { + // set merged acc into state if the merged acc is on state + if (mergeResult != null) { + return windowState + .asyncUpdate(mergeResult, accAndAggResult.f0) + .thenApply(VOID -> accAndAggResult); + } else { + return StateFutureUtils.completedFuture(accAndAggResult); + } + }); + } + + private StateFuture>> collectAccOfSlicesToBeMerged( + Iterable slicesToBeMerged) throws Exception { + List>> futures = new ArrayList<>(); + for (Long slice : slicesToBeMerged) { + futures.add(windowState.asyncValue(slice).thenApply(acc -> Tuple2.of(slice, acc))); + } + return StateFutureUtils.combineAll(futures); + } + + @Override + protected StateFuture sliceStateMergeTarget(long sliceToMerge) throws Exception { + SliceMergeTargetHelper mergeHelper = new SliceMergeTargetHelper(); + return sliceSharedAssigner + .asyncMergeSlices(sliceToMerge, mergeHelper) + .thenApply( + VOID -> { + // the mergeTarget might be null, which means the merging happens in + // memory instead of + // on state, so the slice state to merge into is itself. + if (mergeHelper.getMergeTarget() != null) { + return mergeHelper.getMergeTarget(); + } else { + return sliceToMerge; + } + }); + } + + // ------------------------------------------------------------------------------------------ + + private static final class SliceMergeTargetHelper + implements AsyncMergeCallback>, Serializable { + + private static final long serialVersionUID = 1L; + private static final StateFuture> REUSABLE_FUTURE_RESULT = + StateFutureUtils.completedFuture(null); + private Long mergeTarget = null; + + @Override + public StateFuture> asyncMerge( + @Nullable Long mergeResult, Iterable toBeMerged, Long resultNamespace) { + this.mergeTarget = mergeResult; + return REUSABLE_FUTURE_RESULT; + } + + public Long getMergeTarget() { + return mergeTarget; + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceUnsharedWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceUnsharedWindowAggProcessor.java new file mode 100644 index 0000000000000..4f0ed785f0067 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/asyncwindow/processors/AsyncStateSliceUnsharedWindowAggProcessor.java @@ -0,0 +1,79 @@ +/* + * 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.flink.table.runtime.operators.aggregate.asyncwindow.processors; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceUnsharedAssigner; + +import java.time.ZoneId; + +/** + * A window aggregate processor implementation which works for {@link SliceUnsharedAssigner} with + * async state, e.g. tumbling windows. + */ +public final class AsyncStateSliceUnsharedWindowAggProcessor + extends AbstractAsyncStateSliceWindowAggProcessor { + + private static final long serialVersionUID = 1L; + + public AsyncStateSliceUnsharedWindowAggProcessor( + GeneratedNamespaceAggsHandleFunction genAggsHandler, + AsyncStateWindowBuffer.Factory windowBufferFactory, + SliceUnsharedAssigner sliceAssigner, + TypeSerializer accSerializer, + int indexOfCountStar, + ZoneId shiftTimeZone) { + super( + genAggsHandler, + windowBufferFactory, + sliceAssigner, + accSerializer, + indexOfCountStar, + shiftTimeZone); + } + + @Override + public StateFuture fireWindow(long timerTimestamp, Long windowEnd) throws Exception { + return windowState + .asyncValue(windowEnd) + .thenAccept( + acc -> { + if (acc == null) { + acc = aggregator.createAccumulators(); + } + // the triggered window is an empty window, we shouldn't emit it + if (emptyChecker.apply(acc)) { + return; + } + aggregator.setAccumulators(windowEnd, acc); + RowData aggResult = aggregator.getValue(windowEnd); + collect(ctx.getAsyncKeyContext().getCurrentKey(), aggResult); + }); + } + + @Override + protected StateFuture sliceStateMergeTarget(long sliceToMerge) throws Exception { + return StateFutureUtils.completedFuture(sliceToMerge); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorBuilder.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorBuilder.java index be368f5e6f99b..cc3d0c48ed0d8 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorBuilder.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorBuilder.java @@ -18,29 +18,40 @@ package org.apache.flink.table.runtime.operators.aggregate.window; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateRecordsWindowBuffer; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.buffers.AsyncStateWindowBuffer; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.combines.AsyncStateAggCombiner; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.processors.AsyncStateSliceSharedWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.processors.AsyncStateSliceUnsharedWindowAggProcessor; import org.apache.flink.table.runtime.operators.aggregate.window.buffers.RecordsWindowBuffer; import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; import org.apache.flink.table.runtime.operators.aggregate.window.combines.AggCombiner; import org.apache.flink.table.runtime.operators.aggregate.window.combines.GlobalAggCombiner; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedWindowAggProcessor; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceUnsharedWindowAggProcessor; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.UnsliceWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedSyncStateWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceUnsharedSyncStateWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.UnsliceSyncStateWindowAggProcessor; import org.apache.flink.table.runtime.operators.window.TimeWindow; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowAggOperator; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowProcessor; +import org.apache.flink.table.runtime.operators.window.async.tvf.slicing.AsyncStateSlicingWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.combines.RecordsCombiner; +import org.apache.flink.table.runtime.operators.window.tvf.common.SyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigners.HoppingSliceAssigner; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceUnsharedAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; +import org.apache.flink.util.Preconditions; import java.time.ZoneId; import java.util.function.Supplier; @@ -49,7 +60,7 @@ /** * The {@link WindowAggOperatorBuilder} is used to build a {@link WindowAggOperator} with {@link - * SlicingWindowProcessor} or a {@link UnslicingWindowProcessor}. + * SlicingSyncStateWindowProcessor} or a {@link UnslicingSyncStateWindowProcessor}. * *

  * WindowAggOperatorBuilder.builder()
@@ -70,6 +81,19 @@
  *   .aggregate(genAggsFunction), accTypes)
  *   .build();
  * 
+ * + *

or + * + *

+ * WindowAggOperatorBuilder.builder()
+ *   .inputType(inputType)
+ *   .keyTypes(keyFieldTypes)
+ *   .assigner(UnsliceAssigners.session(rowtimeIndex, Duration.ofSeconds(5)))
+ *   .aggregate(genAggsFunction), accTypes)
+ *   .generatedKeyEqualiser(genKeyEqualiser)
+ *   .enableAsyncState()
+ *   .build();
+ * 
*/ public class WindowAggOperatorBuilder { @@ -84,9 +108,12 @@ public static WindowAggOperatorBuilder builder() { private GeneratedNamespaceAggsHandleFunction generatedAggregateFunction; private GeneratedNamespaceAggsHandleFunction localGeneratedAggregateFunction; private GeneratedNamespaceAggsHandleFunction globalGeneratedAggregateFunction; + private GeneratedRecordEqualiser generatedKeyEqualiser; private int indexOfCountStart = -1; private ZoneId shiftTimeZone; + private boolean enableAsyncState; + public WindowAggOperatorBuilder inputSerializer( AbstractRowDataSerializer inputSerializer) { this.inputSerializer = inputSerializer; @@ -103,6 +130,12 @@ public WindowAggOperatorBuilder keySerializer(PagedTypeSerializer keySe return this; } + public WindowAggOperatorBuilder generatedKeyEqualiser( + GeneratedRecordEqualiser generatedKeyEqualiser) { + this.generatedKeyEqualiser = generatedKeyEqualiser; + return this; + } + public WindowAggOperatorBuilder assigner(WindowAssigner assigner) { this.assigner = assigner; return this; @@ -140,14 +173,27 @@ public WindowAggOperatorBuilder countStarIndex(int indexOfCountStart) { return this; } - public WindowAggOperator build() { + public WindowAggOperatorBuilder enableAsyncState() { + this.enableAsyncState = true; + return this; + } + + public OneInputStreamOperator build() { checkNotNull(assigner); checkNotNull(inputSerializer); checkNotNull(keySerializer); checkNotNull(accSerializer); checkNotNull(generatedAggregateFunction); - final WindowProcessor windowProcessor; + if (enableAsyncState) { + return buildAsyncStateOperator(); + } else { + return buildSyncStateOperator(); + } + } + + private WindowAggOperator buildSyncStateOperator() { + final SyncStateWindowProcessor windowProcessor; if (assigner instanceof SliceAssigner) { windowProcessor = buildSlicingWindowProcessor(); } else { @@ -156,14 +202,24 @@ public WindowAggOperatorBuilder countStarIndex(int indexOfCountStart) { return new WindowAggOperator<>(windowProcessor, assigner.isEventTime()); } - @SuppressWarnings("unchecked") - private SlicingWindowProcessor buildSlicingWindowProcessor() { + private AsyncStateWindowAggOperator buildAsyncStateOperator() { + Preconditions.checkState( + !isGlobalAgg(), "Currently only one-stage window agg supports async state."); + Preconditions.checkState( + assigner instanceof SliceAssigner, + "Currently only slice window supports async state."); - boolean isGlobalAgg = - localGeneratedAggregateFunction != null && globalGeneratedAggregateFunction != null; + checkNotNull(generatedKeyEqualiser); + + final AsyncStateWindowProcessor windowProcessor = + buildAsyncStateSlicingWindowProcessor(); + return new AsyncStateWindowAggOperator<>(windowProcessor, assigner.isEventTime()); + } - RecordsCombiner.Factory combinerFactory; - if (isGlobalAgg) { + @SuppressWarnings("unchecked") + private SlicingSyncStateWindowProcessor buildSlicingWindowProcessor() { + final RecordsCombiner.Factory combinerFactory; + if (isGlobalAgg()) { combinerFactory = new GlobalAggCombiner.Factory( (GeneratedNamespaceAggsHandleFunction) @@ -179,10 +235,10 @@ private SlicingWindowProcessor buildSlicingWindowProcessor() { final WindowBuffer.Factory bufferFactory = new RecordsWindowBuffer.Factory(keySerializer, inputSerializer, combinerFactory); - final SlicingWindowProcessor windowProcessor; + final SlicingSyncStateWindowProcessor windowProcessor; if (assigner instanceof SliceSharedAssigner) { windowProcessor = - new SliceSharedWindowAggProcessor( + new SliceSharedSyncStateWindowAggProcessor( (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction, bufferFactory, (SliceSharedAssigner) assigner, @@ -191,7 +247,7 @@ private SlicingWindowProcessor buildSlicingWindowProcessor() { shiftTimeZone); } else if (assigner instanceof SliceUnsharedAssigner) { windowProcessor = - new SliceUnsharedWindowAggProcessor( + new SliceUnsharedSyncStateWindowAggProcessor( (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction, bufferFactory, (SliceUnsharedAssigner) assigner, @@ -206,12 +262,48 @@ private SlicingWindowProcessor buildSlicingWindowProcessor() { } @SuppressWarnings("unchecked") - private UnsliceWindowAggProcessor buildUnslicingWindowProcessor() { - return new UnsliceWindowAggProcessor( + private UnsliceSyncStateWindowAggProcessor buildUnslicingWindowProcessor() { + return new UnsliceSyncStateWindowAggProcessor( (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction, (UnsliceAssigner) assigner, accSerializer, indexOfCountStart, shiftTimeZone); } + + @SuppressWarnings("unchecked") + private AsyncStateSlicingWindowProcessor buildAsyncStateSlicingWindowProcessor() { + final AsyncStateAggCombiner.Factory combinerFactory = + new AsyncStateAggCombiner.Factory( + (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction); + + final AsyncStateWindowBuffer.Factory bufferFactory = + new AsyncStateRecordsWindowBuffer.Factory( + keySerializer, inputSerializer, combinerFactory, generatedKeyEqualiser); + + if (assigner instanceof SliceSharedAssigner) { + return new AsyncStateSliceSharedWindowAggProcessor( + (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction, + bufferFactory, + (SliceSharedAssigner) assigner, + accSerializer, + indexOfCountStart, + shiftTimeZone); + } else if (assigner instanceof SliceUnsharedAssigner) { + return new AsyncStateSliceUnsharedWindowAggProcessor( + (GeneratedNamespaceAggsHandleFunction) generatedAggregateFunction, + bufferFactory, + (SliceUnsharedAssigner) assigner, + accSerializer, + indexOfCountStart, + shiftTimeZone); + } else { + throw new IllegalArgumentException( + "assigner must be instance of SliceUnsharedAssigner or SliceSharedAssigner."); + } + } + + private boolean isGlobalAgg() { + return localGeneratedAggregateFunction != null && globalGeneratedAggregateFunction != null; + } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceSyncStateWindowAggProcessor.java similarity index 94% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceWindowAggProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceSyncStateWindowAggProcessor.java index 19c727a5525fc..13797f9f71767 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceWindowAggProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSliceSyncStateWindowAggProcessor.java @@ -27,7 +27,7 @@ import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceSharedAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl; import java.time.ZoneId; @@ -35,9 +35,10 @@ import static org.apache.flink.table.runtime.util.TimeWindowUtil.getNextTriggerWatermark; import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; -/** A base implementation of {@link SlicingWindowProcessor} for window aggregate. */ -public abstract class AbstractSliceWindowAggProcessor extends AbstractWindowAggProcessor - implements SlicingWindowProcessor { +/** A base implementation of {@link SlicingSyncStateWindowProcessor} for window aggregate. */ +public abstract class AbstractSliceSyncStateWindowAggProcessor + extends AbstractSyncStateWindowAggProcessor + implements SlicingSyncStateWindowProcessor { protected final WindowBuffer.Factory windowBufferFactory; protected final SliceAssigner sliceAssigner; protected final long windowInterval; @@ -49,7 +50,7 @@ public abstract class AbstractSliceWindowAggProcessor extends AbstractWindowAggP protected transient WindowBuffer windowBuffer; - public AbstractSliceWindowAggProcessor( + public AbstractSliceSyncStateWindowAggProcessor( GeneratedNamespaceAggsHandleFunction genAggsHandler, WindowBuffer.Factory bufferFactory, SliceAssigner sliceAssigner, @@ -69,7 +70,7 @@ public AbstractSliceWindowAggProcessor( } @Override - public void open(Context context) throws Exception { + public void open(SyncStateContext context) throws Exception { super.open(context); this.windowBuffer = windowBufferFactory.create( diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSyncStateWindowAggProcessor.java similarity index 61% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSyncStateWindowAggProcessor.java index e0a59d158b671..219acc03ccabd 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractSyncStateWindowAggProcessor.java @@ -23,122 +23,69 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.state.internal.InternalValueState; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.utils.JoinedRowData; import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; -import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction; -import org.apache.flink.table.runtime.operators.window.tvf.common.ClockService; +import org.apache.flink.table.runtime.operators.window.tvf.common.SyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigners; import org.apache.flink.table.runtime.operators.window.tvf.state.WindowValueState; import java.io.Serializable; import java.time.ZoneId; -import java.util.TimeZone; import java.util.function.Supplier; import static org.apache.flink.util.Preconditions.checkArgument; /** A base class for window aggregate processors. */ -public abstract class AbstractWindowAggProcessor implements WindowProcessor { +public abstract class AbstractSyncStateWindowAggProcessor + extends WindowAggProcessorBase> + implements SyncStateWindowProcessor { private static final long serialVersionUID = 1L; - protected final GeneratedNamespaceAggsHandleFunction genAggsHandler; - - protected final TypeSerializer accSerializer; - - protected final boolean isEventTime; - - protected final ZoneId shiftTimeZone; - - /** The shift timezone is using DayLightSaving time or not. */ - protected final boolean useDayLightSaving; - protected final WindowIsEmptySupplier emptySupplier; // ---------------------------------------------------------------------------------------- - protected transient long currentProgress; - - protected transient WindowProcessor.Context ctx; - - protected transient ClockService clockService; - - protected transient WindowTimerService windowTimerService; - - protected transient NamespaceAggsHandleFunction aggregator; - /** state schema: [key, window, accumulator]. */ protected transient WindowValueState windowState; - protected transient JoinedRowData reuseOutput; - - public AbstractWindowAggProcessor( + public AbstractSyncStateWindowAggProcessor( GeneratedNamespaceAggsHandleFunction genAggsHandler, WindowAssigner sliceAssigner, TypeSerializer accSerializer, boolean isEventTime, int indexOfCountStar, ZoneId shiftTimeZone) { - this.genAggsHandler = genAggsHandler; - this.accSerializer = accSerializer; - this.isEventTime = isEventTime; - this.shiftTimeZone = shiftTimeZone; - this.useDayLightSaving = TimeZone.getTimeZone(shiftTimeZone).useDaylightTime(); + super(genAggsHandler, accSerializer, isEventTime, shiftTimeZone); this.emptySupplier = new WindowIsEmptySupplier(indexOfCountStar, sliceAssigner); } @Override - public void open(WindowProcessor.Context context) throws Exception { - this.ctx = context; - final TypeSerializer namespaceSerializer = createWindowSerializer(); + public void open(SyncStateContext context) throws Exception { + super.open(context); + ValueState state = ctx.getKeyedStateBackend() .getOrCreateKeyedState( - namespaceSerializer, + createWindowSerializer(), new ValueStateDescriptor<>("window-aggs", accSerializer)); this.windowState = new WindowValueState<>((InternalValueState) state); - this.clockService = ClockService.of(ctx.getTimerService()); + } + + @Override + protected final void prepareAggregator() throws Exception { this.aggregator = genAggsHandler.newInstance(ctx.getRuntimeContext().getUserCodeClassLoader()); this.aggregator.open( new PerWindowStateDataViewStore( - ctx.getKeyedStateBackend(), namespaceSerializer, ctx.getRuntimeContext())); - this.reuseOutput = new JoinedRowData(); - this.currentProgress = Long.MIN_VALUE; - this.windowTimerService = getWindowTimerService(); - } - - protected abstract WindowTimerService getWindowTimerService(); - - @Override - public void initializeWatermark(long watermark) { - if (isEventTime) { - currentProgress = watermark; - } - } - - @Override - public void close() throws Exception { - if (aggregator != null) { - aggregator.close(); - } + ctx.getKeyedStateBackend(), + createWindowSerializer(), + ctx.getRuntimeContext())); } - /** - * Send result to downstream. - * - *

The {@link org.apache.flink.types.RowKind} of the results is always {@link - * org.apache.flink.types.RowKind#INSERT}. - * - *

TODO support early fire / late file to produce changelog result. - */ protected void collect(RowData aggResult) { - reuseOutput.replace(ctx.getKeyedStateBackend().getCurrentKey(), aggResult); - ctx.output(reuseOutput); + collect(ctx.getKeyedStateBackend().getCurrentKey(), aggResult); } /** A supplier that returns whether the window is empty. */ diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedSyncStateWindowAggProcessor.java similarity index 94% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedSyncStateWindowAggProcessor.java index 6dd076bf54084..d8c9cf713dff8 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedSyncStateWindowAggProcessor.java @@ -32,17 +32,18 @@ import java.util.Optional; /** - * An window aggregate processor implementation which works for {@link SliceSharedAssigner}, e.g. + * A window aggregate processor implementation which works for {@link SliceSharedAssigner}, e.g. * hopping windows and cumulative windows. */ -public final class SliceSharedWindowAggProcessor extends AbstractSliceWindowAggProcessor +public final class SliceSharedSyncStateWindowAggProcessor + extends AbstractSliceSyncStateWindowAggProcessor implements MergeCallback> { private static final long serialVersionUID = 1L; private final SliceSharedAssigner sliceSharedAssigner; private final SliceMergeTargetHelper mergeTargetHelper; - public SliceSharedWindowAggProcessor( + public SliceSharedSyncStateWindowAggProcessor( GeneratedNamespaceAggsHandleFunction genAggsHandler, WindowBuffer.Factory bufferFactory, SliceSharedAssigner sliceAssigner, @@ -66,7 +67,7 @@ public void fireWindow(long timerTimestamp, Long windowEnd) throws Exception { // we have set accumulator in the merge() method RowData aggResult = aggregator.getValue(windowEnd); if (!emptySupplier.get()) { - // the triggered window is an empty window, we shouldn't emit it + // if the triggered window is an empty window, we shouldn't emit it collect(aggResult); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedSyncStateWindowAggProcessor.java similarity index 90% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedWindowAggProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedSyncStateWindowAggProcessor.java index 20126e24e0169..c4b601cd5e5ab 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedWindowAggProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceUnsharedSyncStateWindowAggProcessor.java @@ -27,13 +27,14 @@ import java.time.ZoneId; /** - * An window aggregate processor implementation which works for {@link SliceUnsharedAssigner}, e.g. + * A window aggregate processor implementation which works for {@link SliceUnsharedAssigner}, e.g. * tumbling windows. */ -public final class SliceUnsharedWindowAggProcessor extends AbstractSliceWindowAggProcessor { +public final class SliceUnsharedSyncStateWindowAggProcessor + extends AbstractSliceSyncStateWindowAggProcessor { private static final long serialVersionUID = 1L; - public SliceUnsharedWindowAggProcessor( + public SliceUnsharedSyncStateWindowAggProcessor( GeneratedNamespaceAggsHandleFunction genAggsHandler, WindowBuffer.Factory windowBufferFactory, SliceUnsharedAssigner sliceAssigner, diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceWindowAggProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceSyncStateWindowAggProcessor.java similarity index 97% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceWindowAggProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceSyncStateWindowAggProcessor.java index 00eda35222e19..88ffe670b5f30 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceWindowAggProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/UnsliceSyncStateWindowAggProcessor.java @@ -37,7 +37,7 @@ import org.apache.flink.table.runtime.operators.window.groupwindow.triggers.Trigger; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner; -import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowTimerServiceImpl; import org.apache.flink.util.Preconditions; import org.apache.flink.util.function.BiConsumerWithException; @@ -53,8 +53,9 @@ * An window aggregate processor implementation which works for {@link UnsliceAssigner}, e.g. * session windows. */ -public class UnsliceWindowAggProcessor extends AbstractWindowAggProcessor - implements UnslicingWindowProcessor { +public class UnsliceSyncStateWindowAggProcessor + extends AbstractSyncStateWindowAggProcessor + implements UnslicingSyncStateWindowProcessor { private final UnsliceAssigner unsliceAssigner; @@ -68,7 +69,7 @@ public class UnsliceWindowAggProcessor extends AbstractWindowAggProcessor genAggsHandler, UnsliceAssigner unsliceAssigner, TypeSerializer accSerializer, @@ -90,7 +91,7 @@ public UnsliceWindowAggProcessor( } @Override - public void open(Context context) throws Exception { + public void open(SyncStateContext context) throws Exception { super.open(context); this.metrics = context.getRuntimeContext().getMetricGroup(); this.windowFunction = diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/WindowAggProcessorBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/WindowAggProcessorBase.java new file mode 100644 index 0000000000000..b3e5fb45fbe40 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/WindowAggProcessorBase.java @@ -0,0 +1,120 @@ +/* + * 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.flink.table.runtime.operators.aggregate.window.processors; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.operators.window.tvf.common.ClockService; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; + +import java.time.ZoneId; +import java.util.TimeZone; + +public abstract class WindowAggProcessorBase> + implements WindowProcessor { + + private static final long serialVersionUID = 1L; + + protected final GeneratedNamespaceAggsHandleFunction genAggsHandler; + + protected final TypeSerializer accSerializer; + + protected final boolean isEventTime; + + protected final ZoneId shiftTimeZone; + + /** The shift timezone is using DayLightSaving time or not. */ + protected final boolean useDayLightSaving; + + // ---------------------------------------------------------------------------------------- + + protected transient long currentProgress; + + protected transient C ctx; + + protected transient ClockService clockService; + + protected transient WindowTimerService windowTimerService; + + protected transient NamespaceAggsHandleFunction aggregator; + + protected transient JoinedRowData reuseOutput; + + public WindowAggProcessorBase( + GeneratedNamespaceAggsHandleFunction genAggsHandler, + TypeSerializer accSerializer, + boolean isEventTime, + ZoneId shiftTimeZone) { + this.genAggsHandler = genAggsHandler; + this.accSerializer = accSerializer; + this.isEventTime = isEventTime; + this.shiftTimeZone = shiftTimeZone; + this.useDayLightSaving = TimeZone.getTimeZone(shiftTimeZone).useDaylightTime(); + } + + @Override + public void open(C context) throws Exception { + this.ctx = context; + this.clockService = ClockService.of(ctx.getTimerService()); + + this.aggregator = + genAggsHandler.newInstance(ctx.getRuntimeContext().getUserCodeClassLoader()); + + this.reuseOutput = new JoinedRowData(); + this.currentProgress = Long.MIN_VALUE; + this.windowTimerService = getWindowTimerService(); + + prepareAggregator(); + } + + @Override + public void initializeWatermark(long watermark) { + if (isEventTime) { + currentProgress = watermark; + } + } + + @Override + public void close() throws Exception { + if (aggregator != null) { + aggregator.close(); + } + } + + protected abstract void prepareAggregator() throws Exception; + + protected abstract WindowTimerService getWindowTimerService(); + + /** + * Send result to downstream. + * + *

The {@link org.apache.flink.types.RowKind} of the results is always {@link + * org.apache.flink.types.RowKind#INSERT}. + * + *

TODO support early fire / late file to produce changelog result. + */ + protected void collect(RowData currentKey, RowData aggResult) { + reuseOutput.replace(currentKey, aggResult); + ctx.output(reuseOutput); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/RowTimeWindowDeduplicateOperatorBuilder.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/RowTimeWindowDeduplicateOperatorBuilder.java index ce152cca74181..98267e2adee03 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/RowTimeWindowDeduplicateOperatorBuilder.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/RowTimeWindowDeduplicateOperatorBuilder.java @@ -22,10 +22,10 @@ import org.apache.flink.table.runtime.operators.aggregate.window.buffers.RecordsWindowBuffer; import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; import org.apache.flink.table.runtime.operators.deduplicate.window.combines.RowTimeDeduplicateRecordsCombiner; -import org.apache.flink.table.runtime.operators.deduplicate.window.processors.RowTimeWindowDeduplicateProcessor; +import org.apache.flink.table.runtime.operators.deduplicate.window.processors.RowTimeSyncStateWindowDeduplicateProcessor; import org.apache.flink.table.runtime.operators.window.tvf.combines.RecordsCombiner; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; @@ -105,8 +105,8 @@ public RowTimeWindowDeduplicateOperatorBuilder windowEndIndex(int windowEndIndex inputSerializer, rowtimeIndex, keepLastRow); final WindowBuffer.Factory bufferFactory = new RecordsWindowBuffer.Factory(keySerializer, inputSerializer, combinerFactory); - final SlicingWindowProcessor windowProcessor = - new RowTimeWindowDeduplicateProcessor( + final SlicingSyncStateWindowProcessor windowProcessor = + new RowTimeSyncStateWindowDeduplicateProcessor( inputSerializer, bufferFactory, windowEndIndex, shiftTimeZone); return new WindowAggOperator<>(windowProcessor, true); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeWindowDeduplicateProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeSyncStateWindowDeduplicateProcessor.java similarity index 94% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeWindowDeduplicateProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeSyncStateWindowDeduplicateProcessor.java index 2898f28e56f58..aa75a9a743391 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeWindowDeduplicateProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/window/processors/RowTimeSyncStateWindowDeduplicateProcessor.java @@ -26,7 +26,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl; import org.apache.flink.table.runtime.operators.window.tvf.state.WindowValueState; @@ -35,7 +35,8 @@ import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; /** A rowtime window deduplicate processor. */ -public final class RowTimeWindowDeduplicateProcessor implements SlicingWindowProcessor { +public final class RowTimeSyncStateWindowDeduplicateProcessor + implements SlicingSyncStateWindowProcessor { private static final long serialVersionUID = 1L; private final WindowBuffer.Factory bufferFactory; @@ -47,7 +48,7 @@ public final class RowTimeWindowDeduplicateProcessor implements SlicingWindowPro private transient long currentProgress; - private transient Context ctx; + private transient SyncStateContext ctx; private transient WindowTimerService windowTimerService; @@ -56,7 +57,7 @@ public final class RowTimeWindowDeduplicateProcessor implements SlicingWindowPro /** state schema: [key, window_end, first/last record]. */ private transient WindowValueState windowState; - public RowTimeWindowDeduplicateProcessor( + public RowTimeSyncStateWindowDeduplicateProcessor( TypeSerializer inputSerializer, WindowBuffer.Factory bufferFactory, int windowEndIndex, @@ -68,7 +69,7 @@ public RowTimeWindowDeduplicateProcessor( } @Override - public void open(Context context) throws Exception { + public void open(SyncStateContext context) throws Exception { this.ctx = context; final LongSerializer namespaceSerializer = LongSerializer.INSTANCE; ValueStateDescriptor valueStateDescriptor = diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java index 807197ae076ae..50f462820cce7 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java @@ -24,10 +24,10 @@ import org.apache.flink.table.runtime.operators.aggregate.window.buffers.RecordsWindowBuffer; import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; import org.apache.flink.table.runtime.operators.rank.window.combines.TopNRecordsCombiner; -import org.apache.flink.table.runtime.operators.rank.window.processors.WindowRankProcessor; +import org.apache.flink.table.runtime.operators.rank.window.processors.SyncStateWindowRankProcessor; import org.apache.flink.table.runtime.operators.window.tvf.combines.RecordsCombiner; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; @@ -149,8 +149,8 @@ public WindowRankOperatorBuilder withEventTime(Boolean isEventTime) { generatedSortKeyComparator, sortKeySelector, inputSerializer, rankEnd); final WindowBuffer.Factory bufferFactory = new RecordsWindowBuffer.Factory(keySerializer, inputSerializer, combinerFactory); - final SlicingWindowProcessor windowProcessor = - new WindowRankProcessor( + final SlicingSyncStateWindowProcessor windowProcessor = + new SyncStateWindowRankProcessor( inputSerializer, generatedSortKeyComparator, sortKeySelector.getProducedType().toSerializer(), diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/SyncStateWindowRankProcessor.java similarity index 96% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/SyncStateWindowRankProcessor.java index 11ac6d524f0a3..ee6aa9a005267 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/SyncStateWindowRankProcessor.java @@ -31,7 +31,7 @@ import org.apache.flink.table.runtime.operators.aggregate.window.buffers.WindowBuffer; import org.apache.flink.table.runtime.operators.rank.TopNBuffer; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowTimerServiceImpl; import org.apache.flink.table.runtime.operators.window.tvf.state.WindowMapState; import org.apache.flink.types.RowKind; @@ -47,7 +47,7 @@ import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; /** A rowtime window rank processor. */ -public final class WindowRankProcessor implements SlicingWindowProcessor { +public final class SyncStateWindowRankProcessor implements SlicingSyncStateWindowProcessor { private static final long serialVersionUID = 1L; private final GeneratedRecordComparator generatedSortKeyComparator; @@ -69,7 +69,7 @@ public final class WindowRankProcessor implements SlicingWindowProcessor { private transient long currentProgress; - private transient Context ctx; + private transient SyncStateContext ctx; private transient WindowTimerService windowTimerService; @@ -81,7 +81,7 @@ public final class WindowRankProcessor implements SlicingWindowProcessor { private transient JoinedRowData reuseOutput; private transient GenericRowData reuseRankRow; - public WindowRankProcessor( + public SyncStateWindowRankProcessor( TypeSerializer inputSerializer, GeneratedRecordComparator genSortKeyComparator, TypeSerializer sortKeySerializer, @@ -103,7 +103,7 @@ public WindowRankProcessor( } @Override - public void open(Context context) throws Exception { + public void open(SyncStateContext context) throws Exception { this.ctx = context; // compile comparator diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/AsyncMergeCallback.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/AsyncMergeCallback.java new file mode 100644 index 0000000000000..7acf4b5d0b076 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/AsyncMergeCallback.java @@ -0,0 +1,57 @@ +/* + * 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.flink.table.runtime.operators.window.async; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.window.MergeCallback; +import org.apache.flink.table.runtime.operators.window.Window; + +import javax.annotation.Nullable; + +/** + * Callback to be used in when merging slices or windows for specifying which slices or windows + * should be merged. + * + *

Different with {@link MergeCallback}, this callback maybe merge slices or windows to async + * state. + * + * @param The type {@link Window} for windows or the type {@link Long} for slices that this + * callback used to merge. + * @param The result type like {@link java.util.Collection} or {@link Iterable} to specify which + * slices or windows should be merged. + */ +public interface AsyncMergeCallback { + + /** + * Specifies that states of the given windows or slices should be merged into the result window + * or slice. + * + * @param mergeResult The resulting merged window or slice, {@code null} if it represents a + * non-state namespace. + * @param toBeMerged Windows or slices that should be merged into one window or slice. + * @param resultNamespace The window or slice used as namespace to get the result from the + * merged accumulator. + * @return f0 is the accumulators after merging, f1 is the result of the aggregation from the + * merged accumulators with this slice end as namespace + */ + StateFuture> asyncMerge( + @Nullable W mergeResult, R toBeMerged, W resultNamespace) throws Exception; +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/combines/AsyncStateRecordsCombiner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/combines/AsyncStateRecordsCombiner.java new file mode 100644 index 0000000000000..27a0ae7a7b0e9 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/combines/AsyncStateRecordsCombiner.java @@ -0,0 +1,69 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.combines; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.WindowAsyncState; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowTimerService; + +import java.io.Serializable; +import java.util.Iterator; + +/** The {@link AsyncStateRecordsCombiner} is used to combine buffered records into async state. */ +@Internal +public interface AsyncStateRecordsCombiner { + + /** + * Combines the buffered data into async state based on the given window. + * + * @param window the window that the buffered data belong to, the window object is reused. + * @param records the buffered data, the iterator and {@link RowData} objects are reused. + */ + StateFuture asyncCombine(Long window, Iterator records) throws Exception; + + /** Release resources allocated by this combine function. */ + void close() throws Exception; + + // ------------------------------------------------------------------------ + + /** A factory that creates a {@link AsyncStateRecordsCombiner}. */ + @FunctionalInterface + interface Factory extends Serializable { + + /** + * Creates a {@link AsyncStateRecordsCombiner} that can combine buffered data into async + * states. + * + * @param runtimeContext the current {@link RuntimeContext} + * @param timerService the service to register event-time and processing-time timers + * @param windowState the window state to flush buffered data into. + * @param isEventTime indicates whether the operator works in event-time or processing-time + * mode, used for register corresponding timers. + */ + AsyncStateRecordsCombiner createRecordsCombiner( + RuntimeContext runtimeContext, + WindowTimerService timerService, + WindowAsyncState windowState, + boolean isEventTime) + throws Exception; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowAggOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowAggOperator.java new file mode 100644 index 0000000000000..30a84a7573381 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowAggOperator.java @@ -0,0 +1,336 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.operators.window.async.tvf.common; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.v2.ListState; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Meter; +import org.apache.flink.metrics.MeterView; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.state.OperatorStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.runtime.state.v2.ListStateDescriptor; +import org.apache.flink.streaming.api.operators.InternalTimer; +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.streaming.api.operators.KeyContext; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.TimestampedCollector; +import org.apache.flink.streaming.api.operators.Triggerable; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.AsyncStateTableStreamOperator; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.AsyncStateKeyContext; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; +import org.apache.flink.table.runtime.util.AsyncStateUtils; + +import java.util.Collections; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A processor that processes elements for windows with async state api. + * + *

Different with {@link WindowAggOperator}, this class mainly handles processing related to + * async state. + * + *

You can see more at {@link WindowAggOperator}. + */ +@Internal +public final class AsyncStateWindowAggOperator extends AsyncStateTableStreamOperator + implements OneInputStreamOperator, Triggerable, KeyContext { + + private static final long serialVersionUID = 1L; + + private static final String LATE_ELEMENTS_DROPPED_METRIC_NAME = "numLateRecordsDropped"; + private static final String LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME = "lateRecordsDroppedRate"; + private static final String WATERMARK_LATENCY_METRIC_NAME = "watermarkLatency"; + + /** The concrete window operator implementation. */ + private final AsyncStateWindowProcessor windowProcessor; + + private final boolean isEventTime; + + // ------------------------------------------------------------------------ + + /** This is used for emitting elements with a given timestamp. */ + private transient TimestampedCollector collector; + + /** The service to register timers. */ + private transient InternalTimerService internalTimerService; + + /** The tracked processing time triggered last time. */ + private transient long lastTriggeredProcessingTime; + + /** The operator state to store watermark. */ + private transient ListState watermarkState; + + // ------------------------------------------------------------------------ + // Metrics + // ------------------------------------------------------------------------ + + private transient Counter numLateRecordsDropped; + private transient Meter lateRecordsDroppedRate; + + private transient Gauge watermarkLatency; + + public AsyncStateWindowAggOperator( + AsyncStateWindowProcessor windowProcessor, boolean isEventTime) { + this.windowProcessor = windowProcessor; + this.isEventTime = isEventTime; + } + + @Override + public void open() throws Exception { + super.open(); + + lastTriggeredProcessingTime = Long.MIN_VALUE; + collector = new TimestampedCollector<>(output); + collector.eraseTimestamp(); + + internalTimerService = + getInternalTimerService( + "window-timers", windowProcessor.createWindowSerializer(), this); + + windowProcessor.open( + new WindowProcessorAsyncStateContext<>( + getContainingTask(), + getContainingTask().getEnvironment().getMemoryManager(), + computeMemorySize(), + internalTimerService, + new AsyncStateKeyContext(this, getAsyncKeyedStateBackend()), + collector, + getRuntimeContext())); + // initialize watermark + windowProcessor.initializeWatermark(currentWatermark); + + // metrics + this.numLateRecordsDropped = metrics.counter(LATE_ELEMENTS_DROPPED_METRIC_NAME); + this.lateRecordsDroppedRate = + metrics.meter( + LATE_ELEMENTS_DROPPED_RATE_METRIC_NAME, + new MeterView(numLateRecordsDropped)); + this.watermarkLatency = + metrics.gauge( + WATERMARK_LATENCY_METRIC_NAME, + () -> { + long watermark = internalTimerService.currentWatermark(); + if (watermark < 0) { + return 0L; + } else { + return internalTimerService.currentProcessingTime() - watermark; + } + }); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + ListStateDescriptor watermarkStateDesc = + new ListStateDescriptor<>("watermark", LongSerializer.INSTANCE); + this.watermarkState = + ((OperatorStateBackend) context.getOperatorStateStore()) + .getUnionListState(watermarkStateDesc); + if (context.isRestored()) { + AtomicLong minWatermark = new AtomicLong(Long.MAX_VALUE); + watermarkState + .asyncGet() + .thenCompose( + its -> + its.onNext( + watermark -> { + minWatermark.set( + Math.min(watermark, minWatermark.get())); + })) + .thenAccept( + VOID -> { + if (minWatermark.get() != Long.MAX_VALUE) { + this.currentWatermark = minWatermark.get(); + } + }); + } + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + this.watermarkState.asyncUpdate(Collections.singletonList(currentWatermark)); + } + + @Override + public void close() throws Exception { + super.close(); + collector = null; + windowProcessor.close(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + RowData inputRow = element.getValue(); + RowData currentKey = (RowData) getCurrentKey(); + windowProcessor + .processElement(currentKey, inputRow) + .thenAccept( + isElementDropped -> { + if (isElementDropped) { + // markEvent will increase numLateRecordsDropped + lateRecordsDroppedRate.markEvent(); + } + }); + } + + @Override + public Watermark preProcessWatermark(Watermark mark) throws Exception { + if (mark.getTimestamp() > currentWatermark) { + // If this is a proctime window, progress should not be advanced by watermark, or it'll + // disturb timer-based processing + if (isEventTime) { + windowProcessor.advanceProgress(null, mark.getTimestamp()); + } + return super.preProcessWatermark(mark); + } else { + return super.preProcessWatermark(new Watermark(currentWatermark)); + } + } + + @Override + public void onEventTime(InternalTimer timer) throws Exception { + onTimer(timer); + } + + @Override + public void onProcessingTime(InternalTimer timer) throws Exception { + StateFuture advanceFuture = AsyncStateUtils.REUSABLE_VOID_STATE_FUTURE; + if (timer.getTimestamp() > lastTriggeredProcessingTime) { + // similar to the watermark advance, + // we need to notify WindowProcessor first to flush buffer into state + lastTriggeredProcessingTime = timer.getTimestamp(); + advanceFuture = + windowProcessor.advanceProgress( + (RowData) getCurrentKey(), timer.getTimestamp()); + // timers registered in advanceProgress() should always be smaller than current timer + // so, it should be safe to trigger current timer straightforwards. + } + advanceFuture.thenAccept(VOID -> onTimer(timer)); + } + + private void onTimer(InternalTimer timer) throws Exception { + W window = timer.getNamespace(); + windowProcessor + .fireWindow(timer.getTimestamp(), window) + .thenAccept(VOID -> windowProcessor.clearWindow(timer.getTimestamp(), window)); + // we don't need to clear window timers, + // because there should only be one timer for each window now, which is current timer. + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + super.prepareSnapshotPreBarrier(checkpointId); + windowProcessor.prepareCheckpoint(); + drainStateRequests(); + } + + /** Context implementation for {@link AsyncStateWindowProcessor.AsyncStateContext}. */ + private static final class WindowProcessorAsyncStateContext + implements AsyncStateWindowProcessor.AsyncStateContext { + + private final Object operatorOwner; + private final MemoryManager memoryManager; + private final long memorySize; + private final InternalTimerService timerService; + private final AsyncStateKeyContext asyncStateKeyContext; + private final Output collector; + private final RuntimeContext runtimeContext; + + private WindowProcessorAsyncStateContext( + Object operatorOwner, + MemoryManager memoryManager, + long memorySize, + InternalTimerService timerService, + AsyncStateKeyContext asyncStateKeyContext, + Output collector, + RuntimeContext runtimeContext) { + this.operatorOwner = operatorOwner; + this.memoryManager = memoryManager; + this.memorySize = memorySize; + this.timerService = timerService; + this.asyncStateKeyContext = checkNotNull(asyncStateKeyContext); + this.collector = checkNotNull(collector); + this.runtimeContext = checkNotNull(runtimeContext); + } + + @Override + public Object getOperatorOwner() { + return operatorOwner; + } + + @Override + public MemoryManager getMemoryManager() { + return memoryManager; + } + + @Override + public long getMemorySize() { + return memorySize; + } + + @Override + public AsyncStateKeyContext getAsyncKeyContext() { + return asyncStateKeyContext; + } + + @Override + public InternalTimerService getTimerService() { + return timerService; + } + + @Override + public void output(RowData result) { + collector.collect(result); + } + + @Override + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } + } + + // ------------------------------------------------------------------------------ + // Visible For Testing + // ------------------------------------------------------------------------------ + + @VisibleForTesting + public Counter getNumLateRecordsDropped() { + return numLateRecordsDropped; + } + + @VisibleForTesting + public Gauge getWatermarkLatency() { + return watermarkLatency; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowProcessor.java new file mode 100644 index 0000000000000..d51b9699bdc89 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/common/AsyncStateWindowProcessor.java @@ -0,0 +1,97 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.common; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.core.state.StateFutureUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.window.async.tvf.state.AsyncStateKeyContext; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; + +import javax.annotation.Nullable; + +/** A processor that processes elements for windows with async state. */ +@Internal +public interface AsyncStateWindowProcessor + extends WindowProcessor> { + + /** + * Process an element with associated key from the input stream. Returns true if this element is + * dropped because of late arrival. + * + * @param key the key associated with the element + * @param element The element to process. + */ + StateFuture processElement(RowData key, RowData element) throws Exception; + + /** + * Advances the progress time, the progress time is watermark if working in event-time mode, or + * current processing time if working in processing-time mode. + * + *

This will potentially flush buffered data into states, because the watermark advancement + * may be in a very small step, but we don't need to flush buffered data for every watermark + * advancement. + * + *

Note: There may be multiple different keys within the buffer. When flushing them to the + * async state, only the async state request for the current key of the operator will be + * returned as a {@link StateFuture}. Requests for async states for other keys will not be + * waited on. + * + * @param currentKey the current key of the operator used to return the result of accessing + * async state associated with the same key. If it is null, it means that the returns of + * asynchronous state requests for all keys will not be awaited. + * @param progress the current progress time + * @return the future of the flush operation about current key if the current key is not null, + * else a {@link StateFutureUtils#completedVoidFuture()} will be returned. + */ + StateFuture advanceProgress(@Nullable RowData currentKey, long progress) throws Exception; + + /** Performs a preparation before checkpoint. This usually flushes buffered data into state. */ + StateFuture prepareCheckpoint() throws Exception; + + /** + * Emit results of the given window. + * + *

Note: the key context has been set. + * + * @param timerTimestamp the fired timestamp + * @param window the window to emit + */ + StateFuture fireWindow(long timerTimestamp, W window) throws Exception; + + /** + * Clear state and resources associated with the given window namespace. + * + *

Note: the key context has been set. + * + * @param timerTimestamp the fired timestamp + * @param window the window to clear + */ + StateFuture clearWindow(long timerTimestamp, W window) throws Exception; + + // ------------------------------------------------------------------------------------------ + + /** Information available in an invocation of methods of {@link AsyncStateWindowProcessor}. */ + interface AsyncStateContext extends Context { + + /** Returns the current {@link AsyncStateKeyContext}. */ + AsyncStateKeyContext getAsyncKeyContext(); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/slicing/AsyncStateSlicingWindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/slicing/AsyncStateSlicingWindowProcessor.java new file mode 100644 index 0000000000000..4aa8b7c9ee42c --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/slicing/AsyncStateSlicingWindowProcessor.java @@ -0,0 +1,60 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.slicing; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedSyncStateWindowAggProcessor; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; + +/** + * The {@link AsyncStateSlicingWindowProcessor} is an optimized processing for aligned windows which + * can apply the slicing optimization. The core idea of slicing optimization is to divide all + * elements from a data stream into a finite number of non-overlapping chunks (a.k.a. slices). + * + *

Concept of Slice

+ * + *

Dividing a window of aligned windows into a finite number of non-overlapping chunks, where the + * chunks are slices. It has the following properties: + * + *

    + *
  • An element must only belong to a single slice. + *
  • Slices are non-overlapping, i.e. S_i and S_j should not have any shared elements if i != j. + *
  • A window is consist of a finite number of slices. + *
+ * + *

The {@link AsyncStateSlicingWindowProcessor} have different implementation for aggregate and + * topk or others. + * + *

The {@link AsyncStateSlicingWindowProcessor} usually leverages the {@link SliceAssigner} to + * assign slices and calculate based on the slices. See {@link + * SliceSharedSyncStateWindowAggProcessor} as an example. + * + *

Note: since {@link AsyncStateSlicingWindowProcessor} leverages slicing optimization for + * aligned windows, therefore, it doesn't support unaligned windows, e.g. session window. + * + *

Similar with {@link SlicingSyncStateWindowProcessor}, this processor is used for async state + * api. + * + *

See more details in {@link WindowAggOperator}. + */ +@Internal +public interface AsyncStateSlicingWindowProcessor extends AsyncStateWindowProcessor {} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/AsyncStateKeyContext.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/AsyncStateKeyContext.java new file mode 100644 index 0000000000000..cf0cfa8c12a54 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/AsyncStateKeyContext.java @@ -0,0 +1,51 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.state; + +import org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperator; +import org.apache.flink.runtime.state.AsyncKeyedStateBackend; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.function.ThrowingRunnable; + +/** Context to switch current key in async state backend. */ +public class AsyncStateKeyContext { + + private final AbstractAsyncStateStreamOperator asyncStateProcessingOperator; + + private final AsyncKeyedStateBackend asyncKeyedStateBackend; + + public AsyncStateKeyContext( + AbstractAsyncStateStreamOperator asyncStateProcessingOperator, + AsyncKeyedStateBackend asyncKeyedStateBackend) { + this.asyncStateProcessingOperator = asyncStateProcessingOperator; + this.asyncKeyedStateBackend = asyncKeyedStateBackend; + } + + public void asyncProcessWithKey(RowData key, ThrowingRunnable processing) { + asyncStateProcessingOperator.asyncProcessWithKey(key, processing); + } + + public AsyncKeyedStateBackend getAsyncKeyedStateBackend() { + return asyncKeyedStateBackend; + } + + public RowData getCurrentKey() { + return (RowData) asyncStateProcessingOperator.getCurrentKey(); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncState.java new file mode 100644 index 0000000000000..fed6277148395 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncState.java @@ -0,0 +1,28 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.state; + +import org.apache.flink.api.common.state.v2.StateFuture; + +/** A base interface for manipulate async state with window namespace. */ +public interface WindowAsyncState { + + /** Removes the value mapped under current key and the given window. */ + StateFuture asyncClear(W window); +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncValueState.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncValueState.java new file mode 100644 index 0000000000000..88f6dd24e1c98 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/async/tvf/state/WindowAsyncValueState.java @@ -0,0 +1,59 @@ +/* + * 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.flink.table.runtime.operators.window.async.tvf.state; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.common.state.v2.ValueState; +import org.apache.flink.runtime.state.v2.internal.InternalValueState; +import org.apache.flink.table.data.RowData; + +import java.io.IOException; + +/** A wrapper of {@link ValueState} which is easier to update based on window namespace. */ +public class WindowAsyncValueState implements WindowAsyncState { + + private final InternalValueState windowState; + + public WindowAsyncValueState(InternalValueState windowState) { + this.windowState = windowState; + } + + @Override + public StateFuture asyncClear(W window) { + windowState.setCurrentNamespace(window); + return windowState.asyncClear(); + } + + /** Returns the current value for the state under current key and the given window. */ + public StateFuture asyncValue(W window) throws IOException { + windowState.setCurrentNamespace(window); + return windowState.asyncValue(); + } + + /** + * Update the state with the given value under current key and the given window. + * + * @param window the window namespace. + * @param value the new value for the state. + */ + public StateFuture asyncUpdate(W window, RowData value) throws IOException { + windowState.setCurrentNamespace(window); + return windowState.asyncUpdate(value); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/SyncStateWindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/SyncStateWindowProcessor.java new file mode 100644 index 0000000000000..f03891973b1e6 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/SyncStateWindowProcessor.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.runtime.operators.window.tvf.common; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.state.KeyedStateBackend; +import org.apache.flink.table.data.RowData; + +/** A processor that processes elements for windows. */ +@Internal +public interface SyncStateWindowProcessor + extends WindowProcessor> { + + /** + * Process an element with associated key from the input stream. Returns true if this element is + * dropped because of late arrival. + * + * @param key the key associated with the element + * @param element The element to process. + */ + boolean processElement(RowData key, RowData element) throws Exception; + + /** + * Advances the progress time, the progress time is watermark if working in event-time mode, or + * current processing time if working in processing-time mode. + * + *

This will potentially flush buffered data into states, because the watermark advancement + * may be in a very small step, but we don't need to flush buffered data for every watermark + * advancement. + * + * @param progress the current progress time + */ + void advanceProgress(long progress) throws Exception; + + /** Performs a preparation before checkpoint. This usually flushes buffered data into state. */ + void prepareCheckpoint() throws Exception; + + /** + * Emit results of the given window. + * + *

Note: the key context has been set. + * + * @param timerTimestamp the fired timestamp + * @param window the window to emit + */ + void fireWindow(long timerTimestamp, W window) throws Exception; + + /** + * Clear state and resources associated with the given window namespace. + * + *

Note: the key context has been set. + * + * @param timerTimestamp the fired timestamp + * @param window the window to clear + */ + void clearWindow(long timerTimestamp, W window) throws Exception; + + // ------------------------------------------------------------------------------------------ + + /** Information available in an invocation of methods of {@link SyncStateWindowProcessor}. */ + interface SyncStateContext extends Context { + + /** Returns the current {@link KeyedStateBackend}. */ + KeyedStateBackend getKeyedStateBackend(); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowAggOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowAggOperator.java index 26c941cf8b3b8..d69c4eaa0c12e 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowAggOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowAggOperator.java @@ -43,8 +43,8 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.operators.TableStreamOperator; -import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingWindowProcessor; -import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SlicingSyncStateWindowProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnslicingSyncStateWindowProcessor; import java.util.Collections; @@ -89,7 +89,8 @@ *

Note: currently, {@link WindowAggOperator} doesn't support early-fire and late-arrival. Thus * late elements (elements belong to emitted windows) will be simply dropped. * - *

See more in {@link SlicingWindowProcessor} and {@link UnslicingWindowProcessor}. + *

See more in {@link SlicingSyncStateWindowProcessor} and {@link + * UnslicingSyncStateWindowProcessor}. */ @Internal public final class WindowAggOperator extends TableStreamOperator @@ -102,7 +103,7 @@ public final class WindowAggOperator extends TableStreamOperator private static final String WATERMARK_LATENCY_METRIC_NAME = "watermarkLatency"; /** The concrete window operator implementation. */ - private final WindowProcessor windowProcessor; + private final SyncStateWindowProcessor windowProcessor; private final boolean isEventTime; @@ -128,7 +129,7 @@ public final class WindowAggOperator extends TableStreamOperator private transient Meter lateRecordsDroppedRate; private transient Gauge watermarkLatency; - public WindowAggOperator(WindowProcessor windowProcessor, boolean isEventTime) { + public WindowAggOperator(SyncStateWindowProcessor windowProcessor, boolean isEventTime) { this.windowProcessor = windowProcessor; this.isEventTime = isEventTime; } @@ -146,7 +147,7 @@ public void open() throws Exception { "window-timers", windowProcessor.createWindowSerializer(), this); windowProcessor.open( - new WindowProcessorContext<>( + new WindowProcessorSyncStateContext<>( getContainingTask(), getContainingTask().getEnvironment().getMemoryManager(), computeMemorySize(), @@ -266,8 +267,9 @@ public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { windowProcessor.prepareCheckpoint(); } - /** Context implementation for {@link WindowProcessor.Context}. */ - private static final class WindowProcessorContext implements WindowProcessor.Context { + /** Context implementation for {@link SyncStateWindowProcessor.SyncStateContext}. */ + private static final class WindowProcessorSyncStateContext + implements SyncStateWindowProcessor.SyncStateContext { private final Object operatorOwner; private final MemoryManager memoryManager; @@ -277,7 +279,7 @@ private static final class WindowProcessorContext implements WindowProcessor. private final Output collector; private final RuntimeContext runtimeContext; - private WindowProcessorContext( + private WindowProcessorSyncStateContext( Object operatorOwner, MemoryManager memoryManager, long memorySize, diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowProcessor.java index 6f221a47fc149..059531e99ae05 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/common/WindowProcessor.java @@ -22,18 +22,30 @@ import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.memory.MemoryManager; -import org.apache.flink.runtime.state.KeyedStateBackend; import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowProcessor; import java.io.Serializable; -/** A processor that processes elements for windows. */ +/** + * A base window processor provides common methods used for {@link SyncStateWindowProcessor} and + * {@link AsyncStateWindowProcessor}. + * + * @param the window type. + * @param the context that provides some information for the window processor. + */ @Internal -public interface WindowProcessor extends Serializable { +public interface WindowProcessor> extends Serializable { /** Initialization method for the function. It is called before the actual working methods. */ - void open(Context context) throws Exception; + void open(C context) throws Exception; + + /** + * The tear-down method of the function. It is called after the last call to the main working + * methods. + */ + void close() throws Exception; /** * Initializes the watermark which restores from state. The method is called after open method @@ -43,56 +55,6 @@ public interface WindowProcessor extends Serializable { */ void initializeWatermark(long watermark); - /** - * Process an element with associated key from the input stream. Returns true if this element is - * dropped because of late arrival. - * - * @param key the key associated with the element - * @param element The element to process. - */ - boolean processElement(RowData key, RowData element) throws Exception; - - /** - * Advances the progress time, the progress time is watermark if working in event-time mode, or - * current processing time if working in processing-time mode. - * - *

This will potentially flush buffered data into states, because the watermark advancement - * may be in a very small step, but we don't need to flush buffered data for every watermark - * advancement. - * - * @param progress the current progress time - */ - void advanceProgress(long progress) throws Exception; - - /** Performs a preparation before checkpoint. This usually flushes buffered data into state. */ - void prepareCheckpoint() throws Exception; - - /** - * Emit results of the given window. - * - *

Note: the key context has been set. - * - * @param timerTimestamp the fired timestamp - * @param window the window to emit - */ - void fireWindow(long timerTimestamp, W window) throws Exception; - - /** - * Clear state and resources associated with the given window namespace. - * - *

Note: the key context has been set. - * - * @param timerTimestamp the fired timestamp - * @param window the window to clear - */ - void clearWindow(long timerTimestamp, W window) throws Exception; - - /** - * The tear-down method of the function. It is called after the last call to the main working - * methods. - */ - void close() throws Exception; - /** Returns the serializer of the window type. */ TypeSerializer createWindowSerializer(); @@ -113,9 +75,6 @@ interface Context { /** Returns the managed memory size can be used by this operator. */ long getMemorySize(); - /** Returns the current {@link KeyedStateBackend}. */ - KeyedStateBackend getKeyedStateBackend(); - /** Returns the current {@link InternalTimerService}. */ InternalTimerService getTimerService(); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigner.java index 00b62e1eb91ed..834af35d30d27 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigner.java @@ -30,7 +30,7 @@ *

Note: {@link SliceAssigner} servers as a base interface. Concrete assigners should implement * interface {@link SliceSharedAssigner} or {@link SliceUnsharedAssigner}. * - * @see SlicingWindowProcessor for more definition of slice. + * @see SlicingSyncStateWindowProcessor for more definition of slice. */ @Internal public interface SliceAssigner extends WindowAssigner { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigners.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigners.java index 14af977c5d3a3..efdbf0a412c38 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigners.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceAssigners.java @@ -19,10 +19,13 @@ package org.apache.flink.table.runtime.operators.window.tvf.slicing; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.runtime.operators.window.MergeCallback; import org.apache.flink.table.runtime.operators.window.TimeWindow; +import org.apache.flink.table.runtime.operators.window.async.AsyncMergeCallback; import org.apache.flink.table.runtime.operators.window.tvf.common.ClockService; import org.apache.flink.util.IterableIterator; import org.apache.flink.util.MathUtils; @@ -36,6 +39,7 @@ import java.util.Iterator; import java.util.List; import java.util.Optional; +import java.util.function.Function; import java.util.function.Supplier; import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; @@ -275,6 +279,16 @@ public void mergeSlices(long sliceEnd, MergeCallback> callb callback.merge(null, toBeMerged); } + @Override + public StateFuture> asyncMergeSlices( + long sliceEnd, AsyncMergeCallback> callback) throws Exception { + // the iterable to list all the slices of the triggered window + Iterable toBeMerged = + new HoppingSlicesIterable(sliceEnd, sliceSize, numSlicesPerWindow); + // null namespace means use heap data views, instead of state data views + return callback.asyncMerge(null, toBeMerged, sliceEnd); + } + @Override public Optional nextTriggerWindow(long windowEnd, Supplier isWindowEmpty) { if (isWindowEmpty.get()) { @@ -284,6 +298,16 @@ public Optional nextTriggerWindow(long windowEnd, Supplier isWind } } + @Override + public Optional nextTriggerWindow( + long windowEnd, RowData acc, Function isWindowEmpty) { + if (isWindowEmpty.apply(acc)) { + return Optional.empty(); + } else { + return Optional.of(windowEnd + sliceSize); + } + } + @Override public String getDescription() { return String.format( @@ -373,20 +397,29 @@ public long getSliceEndInterval() { @Override public void mergeSlices(long sliceEnd, MergeCallback> callback) throws Exception { - long windowStart = getWindowStart(sliceEnd); - long firstSliceEnd = windowStart + step; - if (sliceEnd == firstSliceEnd) { - // if this is the first slice, there is nothing to merge - reuseToBeMergedList.clear(); - } else { - // otherwise, merge the current slice state into the first slice state - reuseToBeMergedList.reset(sliceEnd); - } - callback.merge(firstSliceEnd, reuseToBeMergedList); + prepareReusableMergedList(sliceEnd); + callback.merge(getFirstSliceEnd(sliceEnd), reuseToBeMergedList); + } + + @Override + public StateFuture> asyncMergeSlices( + long sliceEnd, AsyncMergeCallback> callback) throws Exception { + prepareReusableMergedList(sliceEnd); + return callback.asyncMerge(getFirstSliceEnd(sliceEnd), reuseToBeMergedList, sliceEnd); } @Override public Optional nextTriggerWindow(long windowEnd, Supplier isWindowEmpty) { + return nextTriggerWindow(windowEnd); + } + + @Override + public Optional nextTriggerWindow( + long windowEnd, RowData acc, Function isWindowEmpty) { + return nextTriggerWindow(windowEnd); + } + + private Optional nextTriggerWindow(long windowEnd) { long nextWindowEnd = windowEnd + step; long maxWindowEnd = getWindowStart(windowEnd) + maxSize; if (nextWindowEnd > maxWindowEnd) { @@ -402,6 +435,22 @@ public String getDescription() { "CumulativeWindow(maxSize=%dms, step=%dms, offset=%dms)", maxSize, step, offset); } + + private long getFirstSliceEnd(long sliceEnd) { + long windowStart = getWindowStart(sliceEnd); + return windowStart + step; + } + + private void prepareReusableMergedList(long sliceEnd) { + long firstSliceEnd = getFirstSliceEnd(sliceEnd); + if (sliceEnd == firstSliceEnd) { + // if this is the first slice, there is nothing to merge + reuseToBeMergedList.clear(); + } else { + // otherwise, merge the current slice state into the first slice state + reuseToBeMergedList.reset(sliceEnd); + } + } } /** @@ -485,11 +534,23 @@ public void mergeSlices(long sliceEnd, MergeCallback> callb innerSharedAssigner.mergeSlices(sliceEnd, callback); } + @Override + public StateFuture> asyncMergeSlices( + long sliceEnd, AsyncMergeCallback> callback) throws Exception { + return innerSharedAssigner.asyncMergeSlices(sliceEnd, callback); + } + @Override public Optional nextTriggerWindow(long windowEnd, Supplier isWindowEmpty) { return innerSharedAssigner.nextTriggerWindow(windowEnd, isWindowEmpty); } + @Override + public Optional nextTriggerWindow( + long windowEnd, RowData acc, Function isWindowEmpty) { + return innerSharedAssigner.nextTriggerWindow(windowEnd, acc, isWindowEmpty); + } + @Override public long getLastWindowEnd(long sliceEnd) { return innerAssigner.getLastWindowEnd(sliceEnd); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceSharedAssigner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceSharedAssigner.java index 187e65c6f93b5..e74b57365cf03 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceSharedAssigner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SliceSharedAssigner.java @@ -19,10 +19,15 @@ package org.apache.flink.table.runtime.operators.window.tvf.slicing; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedWindowAggProcessor; +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.operators.aggregate.asyncwindow.processors.AsyncStateSliceSharedWindowAggProcessor; import org.apache.flink.table.runtime.operators.window.MergeCallback; +import org.apache.flink.table.runtime.operators.window.async.AsyncMergeCallback; import java.util.Optional; +import java.util.function.Function; import java.util.function.Supplier; /** @@ -35,13 +40,24 @@ public interface SliceSharedAssigner extends SliceAssigner { /** - * Determines which slices (if any) should be merged. + * Determines which slices (if any) should be merged and then merge them. * * @param sliceEnd the triggered slice, identified by end timestamp * @param callback a callback that can be invoked to signal which slices should be merged. */ void mergeSlices(long sliceEnd, MergeCallback> callback) throws Exception; + /** + * Determines which slices (if any) should be merged and then merge them. + * + * @param sliceEnd the triggered slice, identified by end timestamp + * @param callback a callback that can be invoked to signal which slices should be merged. + * @return f0 is the accumulators after merging, f1 is the result of the aggregation from the + * merged accumulators with this slice end as namespace + */ + StateFuture> asyncMergeSlices( + long sliceEnd, AsyncMergeCallback> callback) throws Exception; + /** * Returns the optional end timestamp of next window which should be triggered. Empty if no * following window to trigger for now. @@ -52,11 +68,14 @@ public interface SliceSharedAssigner extends SliceAssigner { * register next window if current window is empty (i.e. no records in current window). That * means we will have one more unnecessary window triggered for hopping windows if no elements * arrives for a key for a long time. We will skip to emit window result for the triggered empty - * window, see {@link SliceSharedWindowAggProcessor#fireWindow(Long)}. + * window, see {@link AsyncStateSliceSharedWindowAggProcessor#fireWindow(Long, Long)}. * * @param windowEnd the current triggered window, identified by end timestamp * @param isWindowEmpty a supplier that can be invoked to get whether the triggered window is * empty (i.e. no records in the window). */ Optional nextTriggerWindow(long windowEnd, Supplier isWindowEmpty); + + Optional nextTriggerWindow( + long windowEnd, RowData acc, Function isWindowEmpty); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingWindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingSyncStateWindowProcessor.java similarity index 62% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingWindowProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingSyncStateWindowProcessor.java index b7a177389377c..fb00ed8a8a4eb 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingWindowProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/slicing/SlicingSyncStateWindowProcessor.java @@ -19,14 +19,14 @@ package org.apache.flink.table.runtime.operators.window.tvf.slicing; import org.apache.flink.annotation.Internal; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedSyncStateWindowAggProcessor; +import org.apache.flink.table.runtime.operators.window.tvf.common.SyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; /** - * The {@link SlicingWindowProcessor} is an optimized processing for aligned windows which can apply - * the slicing optimization. The core idea of slicing optimization is to divide all elements from a - * data stream into a finite number of non-overlapping chunks (a.k.a. slices). + * The {@link SlicingSyncStateWindowProcessor} is an optimized processing for aligned windows which + * can apply the slicing optimization. The core idea of slicing optimization is to divide all + * elements from a data stream into a finite number of non-overlapping chunks (a.k.a. slices). * *

Concept of Slice

* @@ -39,17 +39,17 @@ *
  • A window is consist of a finite number of slices. * * - *

    The {@link SlicingWindowProcessor} have different implementation for aggregate and topk or - * others. + *

    The {@link SlicingSyncStateWindowProcessor} have different implementation for aggregate and + * topk or others. * - *

    The {@link SlicingWindowProcessor} usually leverages the {@link SliceAssigner} to assign - * slices and calculate based on the slices. See {@link SliceSharedWindowAggProcessor} as an - * example. + *

    The {@link SlicingSyncStateWindowProcessor} usually leverages the {@link SliceAssigner} to + * assign slices and calculate based on the slices. See {@link + * SliceSharedSyncStateWindowAggProcessor} as an example. * - *

    Note: since {@link SlicingWindowProcessor} leverages slicing optimization for aligned windows, - * therefore, it doesn't support unaligned windows, e.g. session window. + *

    Note: since {@link SlicingSyncStateWindowProcessor} leverages slicing optimization for aligned + * windows, therefore, it doesn't support unaligned windows, e.g. session window. * *

    See more details in {@link WindowAggOperator}. */ @Internal -public interface SlicingWindowProcessor extends WindowProcessor {} +public interface SlicingSyncStateWindowProcessor extends SyncStateWindowProcessor {} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnsliceAssigner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnsliceAssigner.java index 7bddbdffe2db4..466a294eeb919 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnsliceAssigner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnsliceAssigner.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.operators.aggregate.window.processors.UnsliceWindowAggProcessor; +import org.apache.flink.table.runtime.operators.aggregate.window.processors.UnsliceSyncStateWindowAggProcessor; import org.apache.flink.table.runtime.operators.window.Window; import org.apache.flink.table.runtime.operators.window.groupwindow.assigners.MergingWindowAssigner; import org.apache.flink.table.runtime.operators.window.groupwindow.internal.MergingWindowProcessFunction; @@ -41,7 +41,7 @@ * boundaries are determined based on the messages timestamps and their correlations, some windows * may be merged into one. * - * @see UnsliceWindowAggProcessor for more definition of unslice window. + * @see UnsliceSyncStateWindowAggProcessor for more definition of unslice window. */ @Internal public interface UnsliceAssigner extends WindowAssigner { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingWindowProcessor.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingSyncStateWindowProcessor.java similarity index 70% rename from flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingWindowProcessor.java rename to flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingSyncStateWindowProcessor.java index cc53821f15ad6..2e2e1b367db84 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingWindowProcessor.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/window/tvf/unslicing/UnslicingSyncStateWindowProcessor.java @@ -19,18 +19,19 @@ package org.apache.flink.table.runtime.operators.window.tvf.unslicing; import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.operators.window.tvf.common.SyncStateWindowProcessor; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowProcessor; /** - * The {@link UnslicingWindowProcessor} is an optimized processing for unaligned windows. + * The {@link UnslicingSyncStateWindowProcessor} is an optimized processing for unaligned windows. * - *

    A {@link UnslicingWindowProcessor} usually leverages the {@link UnsliceAssigner} to assign - * slices and calculate based on the window. + *

    A {@link UnslicingSyncStateWindowProcessor} usually leverages the {@link UnsliceAssigner} to + * assign slices and calculate based on the window. * - *

    Note: Currently, the {@link UnslicingWindowProcessor} only support session time window. + *

    Note: Currently, the {@link UnslicingSyncStateWindowProcessor} only support session time + * window. * *

    See more details in {@link WindowAggOperator}. */ @Internal -public interface UnslicingWindowProcessor extends WindowProcessor {} +public interface UnslicingSyncStateWindowProcessor extends SyncStateWindowProcessor {} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/AsyncStateUtils.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/AsyncStateUtils.java new file mode 100644 index 0000000000000..f9813ce8c6bb0 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/util/AsyncStateUtils.java @@ -0,0 +1,32 @@ +/* + * 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.flink.table.runtime.util; + +import org.apache.flink.api.common.state.v2.StateFuture; +import org.apache.flink.core.state.StateFutureUtils; + +/** Utils for processing with async state. */ +public class AsyncStateUtils { + + public static final StateFuture REUSABLE_VOID_STATE_FUTURE = + StateFutureUtils.completedVoidFuture(); + + public static final StateFuture REUSABLE_TRUE_STATE_FUTURE = + StateFutureUtils.completedFuture(true); +} diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java index 28c0fa0e8fe5e..bb975188f97c2 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java @@ -19,10 +19,10 @@ package org.apache.flink.table.runtime.operators.aggregate.window; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigners; import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; @@ -46,13 +46,17 @@ @ExtendWith(ParameterizedTestExtension.class) class SlicingWindowAggOperatorTest extends WindowAggOperatorTestBase { - public SlicingWindowAggOperatorTest(ZoneId shiftTimeZone) { - super(shiftTimeZone); + public SlicingWindowAggOperatorTest(ZoneId shiftTimeZone, boolean enableAsyncState) { + super(shiftTimeZone, enableAsyncState); } - @Parameters(name = "TimeZone = {0}") + @Parameters(name = "TimeZone = {0}, EnableAsyncState = {1}") private static Collection runMode() { - return Arrays.asList(new Object[] {UTC_ZONE_ID}, new Object[] {SHANGHAI_ZONE_ID}); + return Arrays.asList( + new Object[] {UTC_ZONE_ID, false}, + new Object[] {UTC_ZONE_ID, true}, + new Object[] {SHANGHAI_ZONE_ID, false}, + new Object[] {SHANGHAI_ZONE_ID, true}); } @TestTemplate @@ -62,15 +66,8 @@ void testEventTimeHoppingWindows() throws Exception { 2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1)); final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .countStarIndex(1) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, 1); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -159,7 +156,7 @@ void testEventTimeHoppingWindows() throws Exception { ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); - assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(1); + assertThat(getNumLateRecordsDroppedCount(operator)).isEqualTo(1); testHarness.close(); } @@ -170,15 +167,8 @@ void testProcessingTimeHoppingWindows() throws Exception { SliceAssigners.hopping(-1, shiftTimeZone, Duration.ofHours(3), Duration.ofHours(1)); final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .countStarIndex(1) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, 1); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -293,14 +283,8 @@ void testEventTimeCumulativeWindows() throws Exception { 2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1)); final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -398,7 +382,7 @@ void testEventTimeCumulativeWindows() throws Exception { ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); - assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(1); + assertThat(getNumLateRecordsDroppedCount(operator)).isEqualTo(1); testHarness.close(); } @@ -410,14 +394,8 @@ void testProcessingTimeCumulativeWindows() throws Exception { -1, shiftTimeZone, Duration.ofDays(1), Duration.ofHours(8)); final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -545,14 +523,8 @@ void testEventTimeTumblingWindows() throws Exception { SliceAssigners.tumbling(2, shiftTimeZone, Duration.ofSeconds(3)); final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -636,7 +608,7 @@ void testEventTimeTumblingWindows() throws Exception { ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); - assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(2); + assertThat(getNumLateRecordsDroppedCount(operator)).isEqualTo(2); testHarness.close(); } @@ -653,14 +625,8 @@ void testProcessingTimeTumblingWindows() throws Exception { final SlicingSumAndCountAggsFunction aggsFunction = new SlicingSumAndCountAggsFunction(assigner); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -698,6 +664,7 @@ void testProcessingTimeTumblingWindows() throws Exception { epochMills(UTC_ZONE_ID, "1970-01-01T00:00:00"), epochMills(UTC_ZONE_ID, "1970-01-01T05:00:00"))); + testHarness.endInput(); ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); @@ -715,7 +682,7 @@ void testProcessingTimeTumblingWindows() throws Exception { epochMills(UTC_ZONE_ID, "1970-01-01T05:00:00"), epochMills(UTC_ZONE_ID, "1970-01-01T10:00:00"))); - assertThat(operator.getWatermarkLatency().getValue()).isEqualTo(Long.valueOf(0L)); + assertThat(getWatermarkLatency(operator)).isEqualTo(Long.valueOf(0L)); ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); @@ -730,15 +697,7 @@ void testInvalidWindows() { new SlicingSumAndCountAggsFunction(assigner); // hopping window without specifying count star index - assertThatThrownBy( - () -> - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build()) + assertThatThrownBy(() -> buildWindowOperator(assigner, aggsFunction, null)) .hasMessageContaining( "Hopping window requires a COUNT(*) in the aggregate functions."); } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/UnslicingWindowAggOperatorTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/UnslicingWindowAggOperatorTest.java index f5d7c35eebd34..c0661215dcc01 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/UnslicingWindowAggOperatorTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/UnslicingWindowAggOperatorTest.java @@ -19,13 +19,13 @@ package org.apache.flink.table.runtime.operators.aggregate.window; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.keyselector.EmptyRowDataKeySelector; import org.apache.flink.table.runtime.operators.window.TimeWindow; -import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigner; import org.apache.flink.table.runtime.operators.window.tvf.unslicing.UnsliceAssigners; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; @@ -57,13 +57,17 @@ @ExtendWith(ParameterizedTestExtension.class) class UnslicingWindowAggOperatorTest extends WindowAggOperatorTestBase { - UnslicingWindowAggOperatorTest(ZoneId shiftTimeZone) { - super(shiftTimeZone); + UnslicingWindowAggOperatorTest(ZoneId shiftTimeZone, boolean enableAsyncState) { + super(shiftTimeZone, enableAsyncState); } - @Parameters(name = "TimeZone = {0}") + @Parameters(name = "TimeZone = {0}, EnableAsyncState = {1}") private static Collection runMode() { - return Arrays.asList(new Object[] {UTC_ZONE_ID}, new Object[] {SHANGHAI_ZONE_ID}); + return Arrays.asList( + new Object[] {UTC_ZONE_ID, false}, + new Object[] {UTC_ZONE_ID, true}, + new Object[] {SHANGHAI_ZONE_ID, false}, + new Object[] {SHANGHAI_ZONE_ID, true}); } @TestTemplate @@ -73,14 +77,8 @@ void testEventTimeSessionWindows() throws Exception { final UnslicingSumAndCountAggsFunction aggsFunction = new UnslicingSumAndCountAggsFunction(); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -168,7 +166,7 @@ void testEventTimeSessionWindows() throws Exception { ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); - assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(1); + assertThat(getNumLateRecordsDroppedCount(operator)).isEqualTo(1); testHarness.close(); } @@ -180,15 +178,8 @@ void testEventTimeSessionWindowsWithChangelog() throws Exception { final UnslicingSumAndCountAggsFunction aggsFunction = new UnslicingSumAndCountAggsFunction(); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .countStarIndex(1) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, 1); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -297,7 +288,7 @@ void testEventTimeSessionWindowsWithChangelog() throws Exception { ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); - assertThat(operator.getNumLateRecordsDropped().getCount()).isEqualTo(1); + assertThat(getNumLateRecordsDroppedCount(operator)).isEqualTo(1); testHarness.close(); } @@ -309,14 +300,8 @@ void testProcessingTimeSessionWindows() throws Exception { final UnslicingSumAndCountAggsFunction aggsFunction = new UnslicingSumAndCountAggsFunction(); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, null); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -388,7 +373,7 @@ void testProcessingTimeSessionWindows() throws Exception { epochMills(UTC_ZONE_ID, "1970-01-01T00:00:07"), epochMills(UTC_ZONE_ID, "1970-01-01T00:00:11"))); - assertThat(operator.getWatermarkLatency().getValue()).isEqualTo(Long.valueOf(0L)); + assertThat(getWatermarkLatency(operator)).isEqualTo(Long.valueOf(0L)); ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); @@ -401,15 +386,8 @@ void testProcessingTimeSessionWindowsWithChangelog() throws Exception { final UnslicingSumAndCountAggsFunction aggsFunction = new UnslicingSumAndCountAggsFunction(); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer(KEY_SER) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .countStarIndex(1) - .build(); + OneInputStreamOperator operator = + buildWindowOperator(assigner, aggsFunction, 1); OneInputStreamOperatorTestHarness testHarness = createTestHarness(operator); @@ -521,7 +499,7 @@ void testProcessingTimeSessionWindowsWithChangelog() throws Exception { epochMills(UTC_ZONE_ID, "1970-01-01T00:00:07"), epochMills(UTC_ZONE_ID, "1970-01-01T00:00:14"))); - assertThat(operator.getWatermarkLatency().getValue()).isEqualTo(Long.valueOf(0L)); + assertThat(getWatermarkLatency(operator)).isEqualTo(Long.valueOf(0L)); ASSERTER.assertOutputEqualsSorted( "Output was not correct.", expectedOutput, testHarness.getOutput()); testHarness.close(); @@ -545,16 +523,12 @@ void testSessionWindowsWithoutPartitionKey() throws Exception { final EmptyRowDataKeySelector keySelector = EmptyRowDataKeySelector.INSTANCE; final UnslicingSumAndCountAggsFunction aggsFunction = new UnslicingSumAndCountAggsFunction(); - WindowAggOperator operator = - WindowAggOperatorBuilder.builder() - .inputSerializer(INPUT_ROW_SER) - .shiftTimeZone(shiftTimeZone) - .keySerializer( - (PagedTypeSerializer) - keySelector.getProducedType().toSerializer()) - .assigner(assigner) - .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER) - .build(); + OneInputStreamOperator operator = + buildWindowOperator( + assigner, + aggsFunction, + (PagedTypeSerializer) keySelector.getProducedType().toSerializer(), + null); OneInputStreamOperatorTestHarness testHarness = new KeyedOneInputStreamOperatorTestHarness<>( diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorTestBase.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorTestBase.java index 95fbec138c10f..941a0fe486d54 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorTestBase.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/WindowAggOperatorTestBase.java @@ -19,16 +19,23 @@ package org.apache.flink.table.runtime.operators.aggregate.window; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.streaming.util.asyncprocessing.AsyncKeyedOneInputStreamOperatorTestHarness; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; import org.apache.flink.table.runtime.dataview.StateDataViewStore; import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; import org.apache.flink.table.runtime.generated.NamespaceAggsHandleFunction; +import org.apache.flink.table.runtime.generated.RecordEqualiser; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.operators.window.async.tvf.common.AsyncStateWindowAggOperator; import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAggOperator; +import org.apache.flink.table.runtime.operators.window.tvf.common.WindowAssigner; +import org.apache.flink.table.runtime.operators.window.tvf.slicing.SliceAssigner; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.table.runtime.util.GenericRowRecordSortComparator; @@ -41,6 +48,8 @@ import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.utils.HandwrittenSelectorUtil; +import javax.annotation.Nullable; + import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZoneOffset; @@ -48,6 +57,7 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; /** A test base for window aggregate operator. */ @@ -58,8 +68,11 @@ abstract class WindowAggOperatorTestBase { protected final ZoneId shiftTimeZone; - WindowAggOperatorTestBase(ZoneId shiftTimeZone) { + private final boolean enableAsyncState; + + WindowAggOperatorTestBase(ZoneId shiftTimeZone, boolean enableAsyncState) { this.shiftTimeZone = shiftTimeZone; + this.enableAsyncState = enableAsyncState; } /** Get the timestamp in mills by given epoch mills and timezone. */ @@ -78,12 +91,12 @@ protected long localMills(long epochMills) { new RowType.RowField("f1", new IntType()), new RowType.RowField("f2", new TimestampType()))); - protected static final RowDataSerializer INPUT_ROW_SER = new RowDataSerializer(INPUT_ROW_TYPE); + private static final RowDataSerializer INPUT_ROW_SER = new RowDataSerializer(INPUT_ROW_TYPE); - protected static final RowDataSerializer ACC_SER = + private static final RowDataSerializer ACC_SER = new RowDataSerializer(new BigIntType(), new BigIntType()); - protected static final LogicalType[] OUTPUT_TYPES = + private static final LogicalType[] OUTPUT_TYPES = new LogicalType[] { new VarCharType(Integer.MAX_VALUE), new BigIntType(), @@ -92,13 +105,23 @@ protected long localMills(long epochMills) { new BigIntType() }; - protected static final RowDataKeySelector KEY_SELECTOR = + private static final RowDataKeySelector KEY_SELECTOR = HandwrittenSelectorUtil.getRowDataSelector( new int[] {0}, INPUT_ROW_TYPE.getChildren().toArray(new LogicalType[0])); - protected static final PagedTypeSerializer KEY_SER = + private static final PagedTypeSerializer KEY_SER = (PagedTypeSerializer) KEY_SELECTOR.getProducedType().toSerializer(); + private static final GeneratedRecordEqualiser GEN_KEY_EQUALISER = + new GeneratedRecordEqualiser("", "", new Object[0]) { + private static final long serialVersionUID = 1L; + + @Override + public RecordEqualiser newInstance(ClassLoader classLoader) { + return (key1, key2) -> key1.getString(0).equals(key2.getString(0)); + } + }; + protected static final TypeSerializer OUT_SERIALIZER = new RowDataSerializer(OUTPUT_TYPES); @@ -108,10 +131,54 @@ protected long localMills(long epochMills) { // ============================== Util Functions ============================== + protected OneInputStreamOperator buildWindowOperator( + WindowAssigner assigner, + NamespaceAggsHandleFunction aggsFunction, + @Nullable Integer countStarIndex) { + return buildWindowOperator(assigner, aggsFunction, KEY_SER, countStarIndex); + } + + protected OneInputStreamOperator buildWindowOperator( + WindowAssigner assigner, + NamespaceAggsHandleFunction aggsFunction, + PagedTypeSerializer keySerializer, + @Nullable Integer countStarIndex) { + WindowAggOperatorBuilder builder = + WindowAggOperatorBuilder.builder() + .inputSerializer(INPUT_ROW_SER) + .shiftTimeZone(shiftTimeZone) + .keySerializer(keySerializer) + .assigner(assigner) + .aggregate(createGeneratedAggsHandle(aggsFunction), ACC_SER); + if (countStarIndex != null) { + builder.countStarIndex(countStarIndex); + } + // unslice assigner does not support async state yet + if (enableAsyncState && assigner instanceof SliceAssigner) { + builder.generatedKeyEqualiser(GEN_KEY_EQUALISER); + builder.enableAsyncState(); + } + + OneInputStreamOperator operator = builder.build(); + + if (assigner instanceof SliceAssigner) { + assertThat(isAsyncStateOperator(operator)).isEqualTo(enableAsyncState); + } else { + assertThat(isAsyncStateOperator(operator)).isFalse(); + } + + return operator; + } + protected static OneInputStreamOperatorTestHarness createTestHarness( - WindowAggOperator operator) throws Exception { - return new KeyedOneInputStreamOperatorTestHarness<>( - operator, KEY_SELECTOR, KEY_SELECTOR.getProducedType()); + OneInputStreamOperator operator) throws Exception { + if (isAsyncStateOperator(operator)) { + return AsyncKeyedOneInputStreamOperatorTestHarness.create( + operator, KEY_SELECTOR, KEY_SELECTOR.getProducedType()); + } else { + return new KeyedOneInputStreamOperatorTestHarness<>( + operator, KEY_SELECTOR, KEY_SELECTOR.getProducedType()); + } } protected static GeneratedNamespaceAggsHandleFunction createGeneratedAggsHandle( @@ -133,6 +200,39 @@ protected static long epochMills(ZoneId shiftTimeZone, String timestampStr) { return localDateTime.toInstant(zoneOffset).toEpochMilli(); } + protected static long getNumLateRecordsDroppedCount( + OneInputStreamOperator operator) { + if (operator instanceof WindowAggOperator) { + return ((WindowAggOperator) operator) + .getNumLateRecordsDropped() + .getCount(); + } else if (operator instanceof AsyncStateWindowAggOperator) { + return ((AsyncStateWindowAggOperator) operator) + .getNumLateRecordsDropped() + .getCount(); + } else { + throw new IllegalStateException("Unknown operator: " + operator); + } + } + + protected static long getWatermarkLatency(OneInputStreamOperator operator) { + if (operator instanceof WindowAggOperator) { + return ((WindowAggOperator) operator) + .getWatermarkLatency() + .getValue(); + } else if (operator instanceof AsyncStateWindowAggOperator) { + return ((AsyncStateWindowAggOperator) operator) + .getWatermarkLatency() + .getValue(); + } else { + throw new IllegalStateException("Unknown operator: " + operator); + } + } + + private static boolean isAsyncStateOperator(OneInputStreamOperator operator) { + return operator instanceof AsyncStateWindowAggOperator; + } + /** * This performs a {@code SUM(f1), COUNT(f1)}, where f1 is BIGINT type. The return value * contains {@code sum, count, window_start, window_end}.