diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java index 7ca2627c56f08..193dd6f762f15 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupWindowAggregateOperator.java @@ -61,6 +61,8 @@ import static org.apache.flink.fnexecution.v1.FlinkFnApi.GroupWindow.WindowProperty.WINDOW_END; import static org.apache.flink.fnexecution.v1.FlinkFnApi.GroupWindow.WindowProperty.WINDOW_START; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; /** The Python Group Window AggregateFunction operator for the blink planner. */ @Internal @@ -243,15 +245,17 @@ public void emitResult(Tuple2 resultTuple) throws Exception { setCurrentKey(((RowDataSerializer) getKeySerializer()).toBinaryRow(key)); if (timerOperandType == REGISTER_EVENT_TIMER) { - internalTimerService.registerEventTimeTimer(window, timestamp); + internalTimerService.registerEventTimeTimer( + window, toEpochMillsForTimer(timestamp, shiftTimeZone)); } else if (timerOperandType == REGISTER_PROCESSING_TIMER) { internalTimerService.registerProcessingTimeTimer( - window, TimeWindowUtil.toUtcTimestampMills(timestamp, shiftTimeZone)); + window, toEpochMillsForTimer(timestamp, shiftTimeZone)); } else if (timerOperandType == DELETE_EVENT_TIMER) { - internalTimerService.deleteEventTimeTimer(window, timestamp); + internalTimerService.deleteEventTimeTimer( + window, toEpochMillsForTimer(timestamp, shiftTimeZone)); } else if (timerOperandType == DELETE_PROCESSING_TIMER) { internalTimerService.deleteProcessingTimeTimer( - window, TimeWindowUtil.toUtcTimestampMills(timestamp, shiftTimeZone)); + window, toEpochMillsForTimer(timestamp, shiftTimeZone)); } else { throw new RuntimeException( String.format("Unsupported timerOperandType %s.", timerOperandType)); @@ -407,8 +411,7 @@ private void emitTriggerTimerData(InternalTimer timer, byte processingTime reuseTimerData.setField(2, baos.toByteArray()); baos.reset(); - reuseTimerRowData.setLong( - 2, TimeWindowUtil.toUtcTimestampMills(timer.getTimestamp(), shiftTimeZone)); + reuseTimerRowData.setLong(2, toUtcTimestampMills(timer.getTimestamp(), shiftTimeZone)); udfInputTypeSerializer.serialize(reuseTimerRowData, baosWrapper); pythonFunctionRunner.process(baos.toByteArray()); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java index 5efee2d37fdab..2471d82ba5dcb 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperator.java @@ -54,7 +54,6 @@ import org.apache.flink.table.runtime.operators.window.internal.InternalWindowProcessFunction; import org.apache.flink.table.runtime.operators.window.triggers.Trigger; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.runtime.util.TimeWindowUtil; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; @@ -64,6 +63,10 @@ import java.util.LinkedList; import java.util.List; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMills; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; + /** The Stream Arrow Python {@link AggregateFunction} Operator for Group Window Aggregation. */ @Internal public class StreamArrowPythonGroupWindowAggregateFunctionOperator @@ -201,7 +204,7 @@ public void bufferInput(RowData input) throws Exception { } else { timestamp = internalTimerService.currentProcessingTime(); } - timestamp = TimeWindowUtil.toUtcTimestampMills(timestamp, shiftTimeZone); + timestamp = toUtcTimestampMills(timestamp, shiftTimeZone); // Given the timestamp and element, returns the set of windows into which it // should be placed. @@ -308,7 +311,8 @@ private void buildWindow(PlannerNamedWindowProperty[] namedProperties) { */ private boolean isWindowLate(W window) { return windowAssigner.isEventTime() - && (cleanupTime(window) <= internalTimerService.currentWatermark()); + && (toEpochMillsForTimer(cleanupTime(window), shiftTimeZone) + <= internalTimerService.currentWatermark()); } /** @@ -382,7 +386,7 @@ private boolean hasRetractData( * @param window the window whose state to discard */ private void registerCleanupTimer(W window) { - long cleanupTime = cleanupTime(window); + long cleanupTime = toEpochMillsForTimer(cleanupTime(window), shiftTimeZone); if (cleanupTime == Long.MAX_VALUE) { // don't set a GC timer for "end of time" return; @@ -421,11 +425,11 @@ private void setWindowProperty(W currentWindow) { } private long getShiftEpochMills(long utcTimestampMills) { - return TimeWindowUtil.toEpochMills(utcTimestampMills, shiftTimeZone); + return toEpochMills(utcTimestampMills, shiftTimeZone); } private void cleanWindowIfNeeded(W window, long currentTime) throws Exception { - if (currentTime == cleanupTime(window)) { + if (currentTime == toEpochMillsForTimer(cleanupTime(window), shiftTimeZone)) { windowAccumulateData.setCurrentNamespace(window); windowAccumulateData.clear(); windowRetractData.setCurrentNamespace(window); @@ -453,8 +457,7 @@ boolean onElement(RowData row, long timestamp) throws Exception { } boolean onProcessingTime(long time) throws Exception { - return trigger.onProcessingTime( - TimeWindowUtil.toUtcTimestampMills(time, shiftTimeZone), window); + return trigger.onProcessingTime(time, window); } boolean onEventTime(long time) throws Exception { @@ -482,8 +485,7 @@ public MetricGroup getMetricGroup() { @Override public void registerProcessingTimeTimer(long time) { - internalTimerService.registerProcessingTimeTimer( - window, TimeWindowUtil.toEpochMillsForTimer(time, shiftTimeZone)); + internalTimerService.registerProcessingTimeTimer(window, time); } @Override @@ -493,8 +495,7 @@ public void registerEventTimeTimer(long time) { @Override public void deleteProcessingTimeTimer(long time) { - internalTimerService.deleteProcessingTimeTimer( - window, TimeWindowUtil.toEpochMillsForTimer(time, shiftTimeZone)); + internalTimerService.deleteProcessingTimeTimer(window, time); } @Override @@ -502,6 +503,11 @@ public void deleteEventTimeTimer(long time) { internalTimerService.deleteEventTimeTimer(window, time); } + @Override + public ZoneId getShiftTimeZone() { + return shiftTimeZone; + } + @Override public S getPartitionedState(StateDescriptor stateDescriptor) { try { @@ -536,6 +542,11 @@ public long currentWatermark() { throw new RuntimeException("The method currentWatermark should not be called."); } + @Override + public ZoneId getShiftTimeZone() { + return shiftTimeZone; + } + @Override public RowData getWindowAccumulators(W window) { throw new RuntimeException("The method getWindowAccumulators should not be called."); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java index 062fb5227fd97..4a42cdef63f80 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java @@ -70,6 +70,8 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; + /** PassThroughPythonStreamGroupWindowAggregateOperator. */ public class PassThroughPythonStreamGroupWindowAggregateOperator extends PythonStreamGroupWindowAggregateOperator { @@ -296,11 +298,12 @@ private boolean isWindowLate(TimeWindow window) { } private long cleanupTime(TimeWindow window) { + long windowMaxTs = toEpochMillsForTimer(window.maxTimestamp(), shiftTimeZone); if (windowAssigner.isEventTime()) { - long cleanupTime = Math.max(0, window.maxTimestamp() + allowedLateness); - return cleanupTime >= window.maxTimestamp() ? cleanupTime : Long.MAX_VALUE; + long cleanupTime = Math.max(0, windowMaxTs + allowedLateness); + return cleanupTime >= windowMaxTs ? cleanupTime : Long.MAX_VALUE; } else { - return Math.max(0, window.maxTimestamp()); + return Math.max(0, windowMaxTs); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java index 972981b801363..5aa6015ff62ef 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java @@ -34,6 +34,7 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.resolver.ExpressionResolver.ExpressionResolverBuilder; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.TimestampKind; import org.apache.flink.table.types.logical.TimestampType; @@ -52,9 +53,9 @@ import java.util.stream.Stream; import static org.apache.flink.table.expressions.ApiExpressionUtils.localRef; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.canBeTimeAttributeType; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isProctimeAttribute; -import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.supportedWatermarkType; import static org.apache.flink.table.types.utils.DataTypeUtils.replaceLogicalType; /** Default implementation of {@link SchemaResolver}. */ @@ -182,7 +183,7 @@ private List resolveWatermarkSpecs( // validate time attribute final String timeColumn = watermarkSpec.getColumnName(); - validateTimeColumn(timeColumn, inputColumns); + final Column validatedTimeColumn = validateTimeColumn(timeColumn, inputColumns); // resolve watermark expression final ResolvedExpression watermarkExpression; @@ -197,11 +198,20 @@ private List resolveWatermarkSpecs( } validateWatermarkExpression(watermarkExpression.getOutputDataType().getLogicalType()); + if (!(watermarkExpression.getOutputDataType().getLogicalType().getTypeRoot() + == validatedTimeColumn.getDataType().getLogicalType().getTypeRoot())) { + throw new ValidationException( + String.format( + "The watermark output type %s is different from input time filed type %s.", + watermarkExpression.getOutputDataType(), + validatedTimeColumn.getDataType())); + } + return Collections.singletonList( WatermarkSpec.of(watermarkSpec.getColumnName(), watermarkExpression)); } - private void validateTimeColumn(String columnName, List columns) { + private Column validateTimeColumn(String columnName, List columns) { final Optional timeColumn = columns.stream().filter(c -> c.getName().equals(columnName)).findFirst(); if (!timeColumn.isPresent()) { @@ -212,7 +222,7 @@ private void validateTimeColumn(String columnName, List columns) { columns.stream().map(Column::getName).collect(Collectors.toList()))); } final LogicalType timeFieldType = timeColumn.get().getDataType().getLogicalType(); - if (!supportedWatermarkType(timeFieldType) || getPrecision(timeFieldType) != 3) { + if (!canBeTimeAttributeType(timeFieldType) || getPrecision(timeFieldType) != 3) { throw new ValidationException( "Invalid data type of time field for watermark definition. " + "The field must be of type TIMESTAMP(3) or TIMESTAMP_LTZ(3)."); @@ -221,10 +231,11 @@ private void validateTimeColumn(String columnName, List columns) { throw new ValidationException( "A watermark can not be defined for a processing-time attribute."); } + return timeColumn.get(); } private void validateWatermarkExpression(LogicalType watermarkType) { - if (!supportedWatermarkType(watermarkType) || getPrecision(watermarkType) != 3) { + if (!canBeTimeAttributeType(watermarkType) || getPrecision(watermarkType) != 3) { throw new ValidationException( "Invalid data type of expression for watermark definition. " + "The field must be of type TIMESTAMP(3) or TIMESTAMP_LTZ(3)."); @@ -245,13 +256,29 @@ private Column adjustRowtimeAttribute(List watermarkSpecs, Column final boolean hasWatermarkSpec = watermarkSpecs.stream().anyMatch(s -> s.getRowtimeAttribute().equals(name)); if (hasWatermarkSpec && isStreamingMode) { - final TimestampType originalType = (TimestampType) dataType.getLogicalType(); - final LogicalType rowtimeType = - new TimestampType( - originalType.isNullable(), - TimestampKind.ROWTIME, - originalType.getPrecision()); - return column.copy(replaceLogicalType(dataType, rowtimeType)); + switch (dataType.getLogicalType().getTypeRoot()) { + case TIMESTAMP_WITHOUT_TIME_ZONE: + final TimestampType originalType = (TimestampType) dataType.getLogicalType(); + final LogicalType rowtimeType = + new TimestampType( + originalType.isNullable(), + TimestampKind.ROWTIME, + originalType.getPrecision()); + return column.copy(replaceLogicalType(dataType, rowtimeType)); + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final LocalZonedTimestampType timestampLtzType = + (LocalZonedTimestampType) dataType.getLogicalType(); + final LogicalType rowtimeLtzType = + new LocalZonedTimestampType( + timestampLtzType.isNullable(), + TimestampKind.ROWTIME, + timestampLtzType.getPrecision()); + return column.copy(replaceLogicalType(dataType, rowtimeLtzType)); + default: + throw new ValidationException( + "Invalid data type of expression for rowtime definition. " + + "The field must be of type TIMESTAMP(3) or TIMESTAMP_LTZ(3)."); + } } return column; } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java index 7d34c4e552c1a..0dfb68017fbd3 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java @@ -60,6 +60,11 @@ public class SchemaResolutionTest { private static final ResolvedExpression WATERMARK_RESOLVED = new ResolvedExpressionMock(DataTypes.TIMESTAMP(3), () -> WATERMARK_SQL); + private static final String INVALID_WATERMARK_SQL = + "CAST(ts AS TIMESTAMP_LTZ(3)) - INTERVAL '5' SECOND"; + private static final ResolvedExpression INVALID_WATERMARK_RESOLVED = + new ResolvedExpressionMock(DataTypes.TIMESTAMP_LTZ(3), () -> INVALID_WATERMARK_SQL); + private static final String PROCTIME_SQL = "PROCTIME()"; private static final ResolvedExpression PROCTIME_RESOLVED = @@ -81,6 +86,27 @@ public class SchemaResolutionTest { .columnByExpression("proctime", PROCTIME_SQL) .build(); + // the type of ts_ltz is TIMESTAMP_LTZ + private static final String COMPUTED_SQL_WITH_TS_LTZ = "ts_ltz - INTERVAL '60' MINUTE"; + private static final ResolvedExpression COMPUTED_COLUMN_RESOLVED_WITH_TS_LTZ = + new ResolvedExpressionMock(DataTypes.TIMESTAMP_LTZ(3), () -> COMPUTED_SQL_WITH_TS_LTZ); + private static final String WATERMARK_SQL_WITH_TS_LTZ = "ts1 - INTERVAL '5' SECOND"; + private static final ResolvedExpression WATERMARK_RESOLVED_WITH_TS_LTZ = + new ResolvedExpressionMock(DataTypes.TIMESTAMP_LTZ(3), () -> WATERMARK_SQL_WITH_TS_LTZ); + private static final Schema SCHEMA_WITH_TS_LTZ = + Schema.newBuilder() + .primaryKeyNamed("primary_constraint", "id") // out of order + .column("id", DataTypes.INT().notNull()) + .column("counter", DataTypes.INT().notNull()) + .column("payload", "ROW") + .columnByMetadata("topic", DataTypes.STRING(), true) + .columnByExpression( + "ts1", callSql(COMPUTED_SQL_WITH_TS_LTZ)) // out of order API expression + .columnByMetadata("ts_ltz", DataTypes.TIMESTAMP_LTZ(3), "timestamp") + .watermark("ts1", WATERMARK_SQL_WITH_TS_LTZ) + .columnByExpression("proctime", PROCTIME_SQL) + .build(); + @Test public void testSchemaResolution() { final ResolvedSchema expectedSchema = @@ -118,6 +144,44 @@ public void testSchemaResolution() { } } + @Test + public void testSchemaResolutionWithTimestampLtzRowtime() { + final ResolvedSchema expectedSchema = + new ResolvedSchema( + Arrays.asList( + Column.physical("id", DataTypes.INT().notNull()), + Column.physical("counter", DataTypes.INT().notNull()), + Column.physical( + "payload", + DataTypes.ROW( + DataTypes.FIELD("name", DataTypes.STRING()), + DataTypes.FIELD("age", DataTypes.INT()), + DataTypes.FIELD("flag", DataTypes.BOOLEAN()))), + Column.metadata("topic", DataTypes.STRING(), null, true), + Column.computed("ts1", COMPUTED_COLUMN_RESOLVED_WITH_TS_LTZ), + Column.metadata( + "ts_ltz", DataTypes.TIMESTAMP_LTZ(3), "timestamp", false), + Column.computed("proctime", PROCTIME_RESOLVED)), + Collections.singletonList( + WatermarkSpec.of("ts1", WATERMARK_RESOLVED_WITH_TS_LTZ)), + UniqueConstraint.primaryKey( + "primary_constraint", Collections.singletonList("id"))); + + final ResolvedSchema actualStreamSchema = resolveSchema(SCHEMA_WITH_TS_LTZ, true); + { + assertThat(actualStreamSchema, equalTo(expectedSchema)); + assertTrue(isRowtimeAttribute(getType(actualStreamSchema, "ts1"))); + assertTrue(isProctimeAttribute(getType(actualStreamSchema, "proctime"))); + } + + final ResolvedSchema actualBatchSchema = resolveSchema(SCHEMA_WITH_TS_LTZ, false); + { + assertThat(actualBatchSchema, equalTo(expectedSchema)); + assertFalse(isRowtimeAttribute(getType(actualBatchSchema, "ts1"))); + assertTrue(isProctimeAttribute(getType(actualBatchSchema, "proctime"))); + } + } + @Test public void testSchemaResolutionErrors() { @@ -147,6 +211,13 @@ public void testSchemaResolutionErrors() { .build(), "Invalid expression for watermark 'WATERMARK FOR `ts` AS [INVALID]'."); + testError( + Schema.newBuilder() + .column("ts", DataTypes.TIMESTAMP(3)) + .watermark("ts", callSql(INVALID_WATERMARK_SQL)) + .build(), + "The watermark output type TIMESTAMP_LTZ(3) is different from input time filed type TIMESTAMP(3)."); + testError( Schema.newBuilder() .column("ts", DataTypes.TIMESTAMP(3)) @@ -207,6 +278,20 @@ public void testUnresolvedSchemaString() { + " WATERMARK FOR `ts` AS [ts - INTERVAL '5' SECOND],\n" + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED\n" + ")")); + assertThat( + SCHEMA_WITH_TS_LTZ.toString(), + equalTo( + "(\n" + + " `id` INT NOT NULL,\n" + + " `counter` INT NOT NULL,\n" + + " `payload` [ROW],\n" + + " `topic` METADATA VIRTUAL,\n" + + " `ts1` AS [ts_ltz - INTERVAL '60' MINUTE],\n" + + " `ts_ltz` METADATA FROM 'timestamp',\n" + + " `proctime` AS [PROCTIME()],\n" + + " WATERMARK FOR `ts1` AS [ts1 - INTERVAL '5' SECOND],\n" + + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED\n" + + ")")); } @Test @@ -226,6 +311,22 @@ public void testResolvedSchemaString() { + " WATERMARK FOR `ts`: TIMESTAMP(3) AS ts - INTERVAL '5' SECOND,\n" + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED\n" + ")")); + + final ResolvedSchema resolvedSchemaWithTsLtz = resolveSchema(SCHEMA_WITH_TS_LTZ); + assertThat( + resolvedSchemaWithTsLtz.toString(), + equalTo( + "(\n" + + " `id` INT NOT NULL,\n" + + " `counter` INT NOT NULL,\n" + + " `payload` ROW<`name` STRING, `age` INT, `flag` BOOLEAN>,\n" + + " `topic` STRING METADATA VIRTUAL,\n" + + " `ts1` TIMESTAMP_LTZ(3) *ROWTIME* AS ts_ltz - INTERVAL '60' MINUTE,\n" + + " `ts_ltz` TIMESTAMP_LTZ(3) METADATA FROM 'timestamp',\n" + + " `proctime` TIMESTAMP_LTZ(3) NOT NULL *PROCTIME* AS PROCTIME(),\n" + + " WATERMARK FOR `ts1`: TIMESTAMP_LTZ(3) AS ts1 - INTERVAL '5' SECOND,\n" + + " CONSTRAINT `primary_constraint` PRIMARY KEY (`id`) NOT ENFORCED\n" + + ")")); } @Test @@ -345,13 +446,25 @@ private static ResolvedExpression resolveSqlExpression( inputSchema.getFieldDataType("orig_ts").orElse(null), equalTo(DataTypes.TIMESTAMP(3))); return COMPUTED_COLUMN_RESOLVED; + case COMPUTED_SQL_WITH_TS_LTZ: + assertThat( + inputSchema.getFieldDataType("ts_ltz").orElse(null), + equalTo(DataTypes.TIMESTAMP_LTZ(3))); + return COMPUTED_COLUMN_RESOLVED_WITH_TS_LTZ; case WATERMARK_SQL: assertThat( inputSchema.getFieldDataType("ts").orElse(null), equalTo(DataTypes.TIMESTAMP(3))); return WATERMARK_RESOLVED; + case WATERMARK_SQL_WITH_TS_LTZ: + assertThat( + inputSchema.getFieldDataType("ts1").orElse(null), + equalTo(DataTypes.TIMESTAMP_LTZ(3))); + return WATERMARK_RESOLVED_WITH_TS_LTZ; case PROCTIME_SQL: return PROCTIME_RESOLVED; + case INVALID_WATERMARK_SQL: + return INVALID_WATERMARK_RESOLVED; default: throw new UnsupportedOperationException("Unknown SQL expression."); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java index 1ee1d9d0b5981..f87c27f35884d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/Schema.java @@ -455,8 +455,8 @@ public Builder columnByMetadata( * Declares that the given column should serve as an event-time (i.e. rowtime) attribute and * specifies a corresponding watermark strategy as an expression. * - *

The column must be of type {@code TIMESTAMP(3)} and be a top-level column in the - * schema. It may be a computed column. + *

The column must be of type {@code TIMESTAMP(3)} or {@code TIMESTAMP_LTZ(3)} and be a + * top-level column in the schema. It may be a computed column. * *

The watermark generation expression is evaluated by the framework for every record * during runtime. The framework will periodically emit the largest generated watermark. If diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java index 427cdf4c7fccd..e5bbc3e4d787f 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/api/TableSchema.java @@ -30,6 +30,7 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LegacyTypeInformationType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.table.types.utils.TypeConversions; import org.apache.flink.types.Row; @@ -54,8 +55,7 @@ import static org.apache.flink.table.api.DataTypes.FIELD; import static org.apache.flink.table.api.DataTypes.Field; import static org.apache.flink.table.api.DataTypes.ROW; -import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; -import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.canBeTimeAttributeType; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isCompositeType; import static org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo; import static org.apache.flink.table.types.utils.TypeConversions.fromLegacyInfoToDataType; @@ -520,8 +520,8 @@ private static void validateColumnsAndWatermarkSpecs( String.format( "Rowtime attribute '%s' is not defined in schema.", rowtimeAttribute))); - if (!(rowtimeType.getTypeRoot() == TIMESTAMP_WITHOUT_TIME_ZONE - || rowtimeType.getTypeRoot() == TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + if (!(rowtimeType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE + || rowtimeType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE)) { throw new ValidationException( String.format( "Rowtime attribute '%s' must be of type TIMESTAMP or TIMESTAMP_LTZ but is of type '%s'.", @@ -529,8 +529,7 @@ private static void validateColumnsAndWatermarkSpecs( } LogicalType watermarkOutputType = watermark.getWatermarkExprOutputType().getLogicalType(); - if (!(watermarkOutputType.getTypeRoot() == TIMESTAMP_WITHOUT_TIME_ZONE - || watermarkOutputType.getTypeRoot() == TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + if (!canBeTimeAttributeType(watermarkOutputType)) { throw new ValidationException( String.format( "Watermark strategy %s must be of type TIMESTAMP or TIMESTAMP_LTZ but is of type '%s'.", diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java index 31a9df5477eb2..4a81cee1a946d 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/logical/utils/LogicalTypeChecks.java @@ -109,11 +109,13 @@ public static boolean hasFamily(LogicalType logicalType, LogicalTypeFamily famil } public static boolean isTimeAttribute(LogicalType logicalType) { - return logicalType.accept(TIMESTAMP_KIND_EXTRACTOR) != TimestampKind.REGULAR; + return isRowtimeAttribute(logicalType) || isProctimeAttribute(logicalType); } public static boolean isRowtimeAttribute(LogicalType logicalType) { - return logicalType.accept(TIMESTAMP_KIND_EXTRACTOR) == TimestampKind.ROWTIME; + return (hasRoot(logicalType, LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) + || hasRoot(logicalType, LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) + && logicalType.accept(TIMESTAMP_KIND_EXTRACTOR) == TimestampKind.ROWTIME; } public static boolean isProctimeAttribute(LogicalType logicalType) { @@ -122,24 +124,14 @@ public static boolean isProctimeAttribute(LogicalType logicalType) { } public static boolean canBeTimeAttributeType(LogicalType logicalType) { - if (isProctimeAttribute(logicalType) - && logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE) { - return true; - } - if (isRowtimeAttribute(logicalType) - && (logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE - || logicalType.getTypeRoot() - == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + LogicalTypeRoot typeRoot = logicalType.getTypeRoot(); + if (typeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE + || typeRoot == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE) { return true; } return false; } - public static boolean supportedWatermarkType(LogicalType logicalType) { - return logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE - || logicalType.getTypeRoot() == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; - } - /** * Checks if the given type is a composite type. * diff --git a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java index a6faaf7622a67..3ec5fec10b912 100644 --- a/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java +++ b/flink-table/flink-table-common/src/test/java/org/apache/flink/table/api/TableSchemaTest.java @@ -41,8 +41,10 @@ /** Tests for {@link TableSchema}. */ public class TableSchemaTest { - private static final String WATERMARK_EXPRESSION = "now()"; + private static final String WATERMARK_EXPRESSION = "localtimestamp"; + private static final String WATERMARK_EXPRESSION_TS_LTZ = "now()"; private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + private static final DataType WATERMARK_TS_LTZ_DATATYPE = DataTypes.TIMESTAMP_LTZ(3); @Rule public ExpectedException thrown = ExpectedException.none(); @@ -71,7 +73,7 @@ public void testTableSchema() { + " |-- f2: STRING\n" + " |-- f3: BIGINT AS f0 + 1\n" + " |-- f4: BIGINT METADATA FROM 'other.key' VIRTUAL\n" - + " |-- WATERMARK FOR f1.q2: TIMESTAMP(3) AS now()\n"; + + " |-- WATERMARK FOR f1.q2: TIMESTAMP(3) AS localtimestamp\n"; assertEquals(expected, schema.toString()); // test getFieldNames and getFieldDataType @@ -95,6 +97,28 @@ public void testTableSchema() { assertEquals(schema.hashCode(), schema.copy().hashCode()); } + @Test + public void testWatermarkOnTimestampLtz() { + TableSchema tableSchema = + TableSchema.builder() + .field("f0", DataTypes.TIMESTAMP()) + .field( + "f1", + DataTypes.ROW( + DataTypes.FIELD("q1", DataTypes.STRING()), + DataTypes.FIELD("q2", DataTypes.TIMESTAMP_LTZ(3)))) + .watermark("f1.q2", WATERMARK_EXPRESSION_TS_LTZ, WATERMARK_TS_LTZ_DATATYPE) + .build(); + + // test toString() + String expected = + "root\n" + + " |-- f0: TIMESTAMP(6)\n" + + " |-- f1: ROW<`q1` STRING, `q2` TIMESTAMP_LTZ(3)>\n" + + " |-- WATERMARK FOR f1.q2: TIMESTAMP_LTZ(3) AS now()\n"; + assertEquals(expected, tableSchema.toString()); + } + @Test public void testPersistedRowDataType() { final TableSchema schema = diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogSchemaTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogSchemaTable.java index 896cefce3a5bd..08d99599a9add 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogSchemaTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/catalog/CatalogSchemaTable.java @@ -53,8 +53,8 @@ import java.util.List; import java.util.Optional; -import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isProctimeAttribute; import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isRowtimeAttribute; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isTimeAttribute; /** * Represents a wrapper for {@link CatalogBaseTable} in {@link org.apache.calcite.schema.Schema}. @@ -146,7 +146,7 @@ public RelDataType getRowType(RelDataTypeFactory typeFactory) { if (lt instanceof TimestampType && isRowtimeAttribute(lt)) { int precision = ((TimestampType) lt).getPrecision(); fieldDataTypes[i] = DataTypes.TIMESTAMP(precision); - } else if (lt instanceof LocalZonedTimestampType && isProctimeAttribute(lt)) { + } else if (lt instanceof LocalZonedTimestampType && isTimeAttribute(lt)) { int precision = ((LocalZonedTimestampType) lt).getPrecision(); fieldDataTypes[i] = DataTypes.TIMESTAMP_LTZ(precision); } diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerRowtimeAttribute.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerRowtimeAttribute.java index d11f1a6320da0..38020bc91cb86 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerRowtimeAttribute.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/expressions/PlannerRowtimeAttribute.java @@ -20,6 +20,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.TimestampKind; import org.apache.flink.table.types.logical.TimestampType; @@ -44,8 +45,12 @@ public LogicalType getResultType() { LogicalType resultType = reference.getType().get(); if (resultType instanceof TimestampType && ((TimestampType) resultType).getKind() == TimestampKind.ROWTIME) { - // rowtime window + // rowtime window with TIMESTAMP type return new TimestampType(true, TimestampKind.ROWTIME, 3); + } else if (resultType instanceof LocalZonedTimestampType + && ((LocalZonedTimestampType) resultType).getKind() == TimestampKind.ROWTIME) { + // rowtime window with TIMESTAMP_LTZ type + return new LocalZonedTimestampType(true, TimestampKind.ROWTIME, 3); } else if (resultType instanceof BigIntType || resultType instanceof TimestampType) { // batch time window return new TimestampType(3); diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index 86f577b14b797..60c7bb32b5048 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -73,15 +73,17 @@ public static synchronized FlinkSqlOperatorTable instance() { } private static final SqlReturnTypeInference ROWTIME_TYPE_INFERENCE = - createTimeIndicatorReturnType(true); + createTimeIndicatorReturnType(true, false); private static final SqlReturnTypeInference PROCTIME_TYPE_INFERENCE = - createTimeIndicatorReturnType(false); + createTimeIndicatorReturnType(false, true); - private static SqlReturnTypeInference createTimeIndicatorReturnType(boolean isRowTime) { + private static SqlReturnTypeInference createTimeIndicatorReturnType( + boolean isRowTime, boolean isTimestampLtz) { return ReturnTypes.explicit( factory -> { if (isRowTime) { - return ((FlinkTypeFactory) factory).createRowtimeIndicatorType(false); + return ((FlinkTypeFactory) factory) + .createRowtimeIndicatorType(false, isTimestampLtz); } else { return ((FlinkTypeFactory) factory).createProctimeIndicatorType(false); } @@ -117,7 +119,11 @@ public void lookupOperatorOverloads( SqlFunctionCategory.TIMEDATE, false); - /** Function used to access a event time attribute from MATCH_RECOGNIZE. */ + /** + * Function used to access a event time attribute with TIMESTAMP or TIMESTAMP_LTZ type from + * MATCH_RECOGNIZE, for TIMESTAMP_LTZ type, we rewrite the return type in + * [org.apache.flink.table.planner.calcite.RelTimeIndicatorConverter]. + */ public static final SqlFunction MATCH_ROWTIME = new CalciteSqlFunction( "MATCH_ROWTIME", diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java index d808960445d9c..a3712aca7f8dc 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonDeserializer.java @@ -90,12 +90,15 @@ private RelDataType deserialize(JsonNode jsonNode, FlinkDeserializationContext c ObjectNode objectNode = (ObjectNode) jsonNode; if (objectNode.has(FIELD_NAME_TIMESTAMP_KIND)) { boolean nullable = objectNode.get(FIELD_NAME_NULLABLE).booleanValue(); + String typeName = objectNode.get(FIELD_NAME_TYPE_NAME).textValue(); + boolean isTimestampLtz = + SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE.name().equals(typeName); TimestampKind timestampKind = TimestampKind.valueOf( objectNode.get(FIELD_NAME_TIMESTAMP_KIND).asText().toUpperCase()); switch (timestampKind) { case ROWTIME: - return typeFactory.createRowtimeIndicatorType(nullable); + return typeFactory.createRowtimeIndicatorType(nullable, isTimestampLtz); case PROCTIME: return typeFactory.createProctimeIndicatorType(nullable); default: diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerializer.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerializer.java index d644e7749d83f..288f5b3ca01b2 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerializer.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeJsonSerializer.java @@ -84,6 +84,8 @@ private void serialize(RelDataType relDataType, JsonGenerator gen) throws IOExce timeIndicatorType.isEventTime() ? TimestampKind.ROWTIME.name() : TimestampKind.PROCTIME.name()); + gen.writeStringField( + FIELD_NAME_TYPE_NAME, timeIndicatorType.originalType().getSqlTypeName().name()); gen.writeBooleanField(FIELD_NAME_NULLABLE, relDataType.isNullable()); } else if (relDataType instanceof StructuredRelDataType) { StructuredRelDataType structuredType = (StructuredRelDataType) relDataType; diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java index 7e7af6ebcf95d..af9ee1def7249 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink; import org.apache.flink.table.runtime.typeutils.TypeCheckUtils; import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.TimestampType; @@ -80,7 +81,18 @@ protected RowType checkAndConvertInputTypeIfNeeded(RowType inputRowType) { } else if (rowtimeFieldIndices.size() == 1) { LogicalType[] convertedFieldTypes = inputRowType.getChildren().stream() - .map(t -> TypeCheckUtils.isRowTime(t) ? new TimestampType(3) : t) + .map( + t -> { + if (TypeCheckUtils.isRowTime(t)) { + if (TypeCheckUtils.isTimestampWithLocalZone(t)) { + return new LocalZonedTimestampType(3); + } else { + return new TimestampType(3); + } + } else { + return t; + } + }) .toArray(LogicalType[]::new); return RowType.of( diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java index 3a618cf8b3c94..f333255e53ae9 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java @@ -145,7 +145,8 @@ protected Transformation translateToPlanInternal(PlannerBase planner) { sliceAssigner, (PagedTypeSerializer) selector.getProducedType().toSerializer(), new RowDataSerializer(inputRowType), - generatedAggsHandler); + generatedAggsHandler, + shiftTimeZone); return ExecNodeUtil.createOneInputTransformation( inputTransform, diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java index a13705f5cb095..74e0e1450a249 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java @@ -60,7 +60,6 @@ import org.apache.flink.table.runtime.typeutils.TypeCheckUtils; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimestampType; import org.apache.flink.util.MathUtils; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -82,6 +81,7 @@ import java.util.List; import java.util.Optional; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.getPrecision; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -255,7 +255,7 @@ private Transformation translateOrder( if (TypeCheckUtils.isRowTime(timeOrderFieldType)) { // copy the rowtime field into the StreamRecord timestamp field - int precision = ((TimestampType) timeOrderFieldType).getPrecision(); + int precision = getPrecision(timeOrderFieldType); Transformation transform = new OneInputTransformation<>( inputTransform, diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanAcrossCalcRule.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanAcrossCalcRule.java index 21b4134d7920d..18539f08e205e 100644 --- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanAcrossCalcRule.java +++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanAcrossCalcRule.java @@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexProgram; import org.apache.calcite.rex.RexProgramBuilder; import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.sql.type.SqlTypeName; import java.util.List; import java.util.stream.Collectors; @@ -99,11 +100,13 @@ public RexNode visitInputRef(RexInputRef inputRef) { FlinkTypeFactory typeFactory = ShortcutUtils.unwrapTypeFactory(calc); RexBuilder builder = call.builder().getRexBuilder(); - // cast timestamp type to rowtime type. + // cast timestamp/timestamp_ltz type to rowtime type. RexNode newRowTimeColumn = builder.makeReinterpretCast( typeFactory.createRowtimeIndicatorType( - rowTimeColumn.getType().isNullable()), + rowTimeColumn.getType().isNullable(), + rowTimeColumn.getType().getSqlTypeName() + == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE), rowTimeColumn, null); diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala index 9d88937ff2323..24a5b50e58145 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala @@ -154,7 +154,7 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE => val timestampType = t.asInstanceOf[TimestampType] timestampType.getKind match { - case TimestampKind.ROWTIME => createRowtimeIndicatorType(true) + case TimestampKind.ROWTIME => createRowtimeIndicatorType(t.isNullable, false) case TimestampKind.REGULAR => createSqlType(TIMESTAMP, timestampType.getPrecision) case TimestampKind.PROCTIME => throw new TableException( s"Processing time indicator only supports" + @@ -164,7 +164,8 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) case LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE => val lzTs = t.asInstanceOf[LocalZonedTimestampType] lzTs.getKind match { - case TimestampKind.PROCTIME => createProctimeIndicatorType(true) + case TimestampKind.PROCTIME => createProctimeIndicatorType(t.isNullable) + case TimestampKind.ROWTIME => createRowtimeIndicatorType(t.isNullable, true) case TimestampKind.REGULAR => createSqlType(TIMESTAMP_WITH_LOCAL_TIME_ZONE, lzTs.getPrecision) } @@ -196,9 +197,15 @@ class FlinkTypeFactory(typeSystem: RelDataTypeSystem) /** * Creates a indicator type for event-time, but with similar properties as SQL timestamp. */ - def createRowtimeIndicatorType(isNullable: Boolean): RelDataType = { - val originalType = createFieldTypeFromLogicalType(new TimestampType(isNullable, 3)) - canonize(new TimeIndicatorRelDataType( + def createRowtimeIndicatorType(isNullable: Boolean, isTimestampLtz: Boolean): RelDataType = { + val originalType = if (isTimestampLtz) { + createFieldTypeFromLogicalType(new LocalZonedTimestampType(isNullable, 3)) + } else { + createFieldTypeFromLogicalType(new TimestampType(isNullable, 3)) + } + + canonize( + new TimeIndicatorRelDataType( getTypeSystem, originalType.asInstanceOf[BasicSqlType], isNullable, @@ -546,9 +553,7 @@ object FlinkTypeFactory { case TIMESTAMP_WITH_LOCAL_TIME_ZONE if relDataType.isInstanceOf[TimeIndicatorRelDataType] => val indicator = relDataType.asInstanceOf[TimeIndicatorRelDataType] if (indicator.isEventTime) { - throw new TableException(s"Event time indicator only supports" + - s" TimestampType now, but actual is LocalZonedTimestampType." + - s" This is a bug in planner, please file an issue.") + new LocalZonedTimestampType(true, TimestampKind.ROWTIME, 3) } else { new LocalZonedTimestampType(true, TimestampKind.PROCTIME, 3) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala index 54899123a1f17..e0b0648107ac9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.scala @@ -19,14 +19,14 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.planner.calcite.FlinkTypeFactory._ +import org.apache.flink.table.planner.calcite.FlinkTypeFactory.{isTimeIndicatorType, _} import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.calcite._ import org.apache.flink.table.planner.plan.schema.TimeIndicatorRelDataType import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil import org.apache.flink.table.planner.plan.utils.WindowUtil.groupingContainsWindowStartEnd -import org.apache.flink.table.types.logical.{LocalZonedTimestampType, TimestampType} +import org.apache.flink.table.types.logical.{LocalZonedTimestampType, TimestampKind, TimestampType} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core._ import org.apache.calcite.rel.hint.RelHint @@ -39,6 +39,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable.FINAL import org.apache.flink.table.planner.plan.nodes.hive.LogicalDistribution import java.util.{Collections => JCollections} + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -88,9 +89,17 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { val isNoLongerTimeIndicator : String => Boolean = fieldName => measures.get(fieldName).exists(r => !FlinkTypeFactory.isTimeIndicatorType(r.getType)) + // decide the MATCH_ROWTIME() return type is TIMESTAMP or TIMESTAMP_LTZ, if it is TIMESTAMP_LTZ, + // we need to materialize the output type of LogicalMatch node to TIMESTAMP_LTZ too. + val isLtzRowtimeIndicator = measures.exists { + case (name, node) => FlinkTypeFactory.isTimestampLtzIndicatorType(node.getType) && + FlinkTypeFactory.isRowtimeIndicatorType(node.getType) + } + // materialize all output types val outputType = materializerUtils.getRowTypeWithoutIndicators( matchRel.getRowType, + isLtzRowtimeIndicator, isNoLongerTimeIndicator) LogicalMatch.create( @@ -329,33 +338,44 @@ class RelTimeIndicatorConverter(rexBuilder: RexBuilder) extends RelShuttle { val head = inputTypes.head.getFieldList.map(_.getType) - val isValid = inputTypes.forall { t => + inputTypes.foreach { t => val fieldTypes = t.getFieldList.map(_.getType) - fieldTypes.zip(head).forall { case (l, r) => - // check if time indicators match - if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { - val leftTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime - val rightTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime - leftTime == rightTime - } - // one side is not an indicator - else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { - false - } - // uninteresting types - else { - true - } + fieldTypes.zip(head).foreach{ case (l, r) => + validateUnionPair(l, r) + } + } + + setOp.copy(setOp.getTraitSet, inputs, setOp.all) + } + + private def validateUnionPair(l: RelDataType, r: RelDataType): Unit = { + val exceptionMsg = + s"Union fields with time attributes requires same types, but the types are %s and %s." + // check if time indicators match + val isValid = if (isTimeIndicatorType(l) && isTimeIndicatorType(r)) { + val leftIsEventTime = l.asInstanceOf[TimeIndicatorRelDataType].isEventTime + val rightIsEventTime = r.asInstanceOf[TimeIndicatorRelDataType].isEventTime + if (leftIsEventTime && rightIsEventTime) { + //rowtime must have same type + isTimestampLtzIndicatorType(l) == isTimestampLtzIndicatorType(r) + } else { + leftIsEventTime == rightIsEventTime } } + // one side is not an indicator + else if (isTimeIndicatorType(l) || isTimeIndicatorType(r)) { + false + } + // uninteresting types + else { + true + } if (!isValid) { throw new ValidationException( - "Union fields with time attributes have different types.") + String.format(exceptionMsg, l.toString, r.toString)) } - - setOp.copy(setOp.getTraitSet, inputs, setOp.all) } private def gatherIndicesToMaterialize(aggregate: Aggregate, input: RelNode): Set[Int] = { @@ -624,13 +644,18 @@ class RexTimeIndicatorMaterializer( private val input: Seq[RelDataType]) extends RexShuttle { + private def rowtime(isNullable: Boolean, isTimestampLtz: Boolean = false): RelDataType = { + rexBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory] + .createRowtimeIndicatorType(isNullable, isTimestampLtz) + } + override def visitInputRef(inputRef: RexInputRef): RexNode = { // reference is interesting if (isTimeIndicatorType(inputRef.getType)) { val resolvedRefType = input(inputRef.getIndex) // input is a valid time indicator if (isTimeIndicatorType(resolvedRefType)) { - inputRef + rexBuilder.makeInputRef(resolvedRefType, inputRef.getIndex) } // input has been materialized else { @@ -663,7 +688,7 @@ class RexTimeIndicatorMaterializer( case FlinkSqlOperatorTable.SESSION_OLD | FlinkSqlOperatorTable.HOP_OLD | FlinkSqlOperatorTable.TUMBLE_OLD => - updatedCall.getOperands.toList + updatedCall.getOperands.toList case _ => updatedCall.getOperands.map { o => @@ -695,10 +720,24 @@ class RexTimeIndicatorMaterializer( // All calls in MEASURES and DEFINE are wrapped with FINAL/RUNNING, therefore // we should treat FINAL(MATCH_ROWTIME) and FINAL(MATCH_PROCTIME) as a time attribute - // extraction + // extraction. The type of FINAL(MATCH_ROWTIME) is inferred by first operand's type, + // the initial type of MATCH_ROWTIME is TIMESTAMP(3) *ROWTIME*, it may be rewrote, + // so we re case FINAL if updatedCall.getOperands.size() == 1 && isMatchTimeIndicator(updatedCall.getOperands.get(0)) => - updatedCall + val rowtimeType = updatedCall.getOperands.get(0).getType + rexBuilder.makeCall(rowtimeType, updatedCall.getOperator, updatedCall.getOperands) + + // MATCH_ROWTIME() is a no-args function, it can own two kind of return types based + // on the rowtime attribute type of its input, we rewrite the return type here + case FlinkSqlOperatorTable.MATCH_ROWTIME if isTimeIndicatorType(updatedCall.getType) => + val rowtimeType = input.filter(isTimeIndicatorType).head + rexBuilder.makeCall( + rowtime( + updatedCall.getType.isNullable, + isTimestampLtzIndicatorType(rowtimeType)), + updatedCall.getOperator, + materializedOperands) // do not modify window time attributes case FlinkSqlOperatorTable.TUMBLE_ROWTIME | @@ -707,7 +746,6 @@ class RexTimeIndicatorMaterializer( FlinkSqlOperatorTable.HOP_PROCTIME | FlinkSqlOperatorTable.SESSION_ROWTIME | FlinkSqlOperatorTable.SESSION_PROCTIME | - FlinkSqlOperatorTable.MATCH_ROWTIME | FlinkSqlOperatorTable.MATCH_PROCTIME // since we materialize groupings on time indicators, // we cannot check the operands anymore but the return type at least @@ -769,6 +807,7 @@ class RexTimeIndicatorMaterializerUtils(rexBuilder: RexBuilder) { def getRowTypeWithoutIndicators( relType: RelDataType, + isLtzRowtimeIndicator: Boolean, shouldMaterialize: String => Boolean): RelDataType = { val outputTypeBuilder = rexBuilder .getTypeFactory @@ -776,10 +815,28 @@ class RexTimeIndicatorMaterializerUtils(rexBuilder: RexBuilder) { .builder() relType.getFieldList.asScala.zipWithIndex.foreach { case (field, idx) => - if (isTimeIndicatorType(field.getType) && shouldMaterialize(field.getName)) { - outputTypeBuilder.add( - field.getName, - timestamp(field.getType.isNullable, isTimestampLtzIndicatorType(field.getType))) + if (isTimeIndicatorType(field.getType)) { + val convertedTimeIndicatorFieldType = if (isLtzRowtimeIndicator) { + rexBuilder + .getTypeFactory + .asInstanceOf[FlinkTypeFactory] + .createFieldTypeFromLogicalType( + new LocalZonedTimestampType(field.getType.isNullable, TimestampKind.ROWTIME, 3)) + } else { + field.getType + } + + if (shouldMaterialize(field.getName)) { + outputTypeBuilder.add( + field.getName, + timestamp( + convertedTimeIndicatorFieldType.isNullable, + isTimestampLtzIndicatorType(convertedTimeIndicatorFieldType))) + } else { + outputTypeBuilder.add( + field.getName, + convertedTimeIndicatorFieldType) + } } else { outputTypeBuilder.add(field.getName, field.getType) } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala index 74114f45ce44f..ce8eb4232272e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala @@ -804,7 +804,7 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) generateProctimeTimestamp(ctx, contextTerm) case STREAMRECORD_TIMESTAMP => - generateRowtimeAccess(ctx, contextTerm) + generateRowtimeAccess(ctx, contextTerm, false) case _: SqlThrowExceptionFunction => val nullValue = generateNullLiteral(resultType, nullCheck = true) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index a0351ae0d5708..47465678bb0b3 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -509,8 +509,13 @@ object GenerateUtils { def generateRowtimeAccess( ctx: CodeGeneratorContext, - contextTerm: String): GeneratedExpression = { - val resultType = new TimestampType(true, TimestampKind.ROWTIME, 3) + contextTerm: String, + isTimestampLtz: Boolean): GeneratedExpression = { + val resultType = if (isTimestampLtz) { + new LocalZonedTimestampType(true, TimestampKind.ROWTIME, 3) + } else { + new TimestampType(true, TimestampKind.ROWTIME, 3) + } val resultTypeTerm = primitiveTypeTermForType(resultType) val Seq(resultTerm, nullTerm, timestamp) = ctx.addReusableLocalVariables( (resultTypeTerm, "result"), diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala index a1a9ec04fe197..673d25a71a5e1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala @@ -375,7 +375,10 @@ class MatchCodeGenerator( ctx.nullCheck) case MATCH_ROWTIME => - generateRowtimeAccess(ctx, contextTerm) + generateRowtimeAccess( + ctx, + contextTerm, + FlinkTypeFactory.isTimestampLtzIndicatorType(call.getType)) case PROCTIME_MATERIALIZE => // override proctime materialize code generation diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala index 074a7d8946aa0..3f21e88272640 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenerator.scala @@ -48,9 +48,10 @@ object WatermarkGeneratorCodeGenerator { contextTerm: Option[String] = None): GeneratedWatermarkGenerator = { // validation val watermarkOutputType = FlinkTypeFactory.toLogicalType(watermarkExpr.getType) - if (watermarkOutputType.getTypeRoot != LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE) { + if (!(watermarkOutputType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE || + watermarkOutputType.getTypeRoot == LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { throw new CodeGenException( - "WatermarkGenerator only accepts output data type of TIMESTAMP," + + "WatermarkGenerator only accepts output data type of TIMESTAMP or TIMESTAMP_LTZ," + " but is " + watermarkOutputType) } val funcName = newName("WatermarkGenerator") diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WatermarkAssigner.scala index 459981afad944..cf76fa1965a27 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WatermarkAssigner.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WatermarkAssigner.scala @@ -26,6 +26,8 @@ import org.apache.calcite.rex.RexNode import java.util +import org.apache.calcite.sql.`type`.SqlTypeName + import scala.collection.JavaConversions._ /** @@ -45,7 +47,9 @@ abstract class WatermarkAssigner( val newFieldList = inputRowType.getFieldList.map { f => if (f.getIndex == rowtimeFieldIndex) { - val rowtimeIndicatorType = typeFactory.createRowtimeIndicatorType(f.getType.isNullable) + val rowtimeIndicatorType = typeFactory.createRowtimeIndicatorType( + f.getType.isNullable, + f.getType.getSqlTypeName == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE) new RelDataTypeFieldImpl(f.getName, f.getIndex, rowtimeIndicatorType) } else { f diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala index 3dd804ea20d50..413a3d655e433 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/LogicalCorrelateToJoinFromTemporalTableRule.scala @@ -286,6 +286,14 @@ abstract class LogicalCorrelateToJoinFromGeneralTemporalTableRule( s" primary key and row time attribute in versioned table," + s" but no row time attribute can be found.") } + + if (snapshotTimeInputRef.getType.getSqlTypeName + != rightTimeInputRef.get.getType.getSqlTypeName) { + throw new ValidationException( + String.format("Event-Time Temporal Table Join requires same rowtime" + + " type in left table and versioned table, but the rowtime types are %s and %s.", + snapshotTimeInputRef.getType.toString, rightTimeInputRef.get.getType.toString)) + } // Deal primary key in TemporalJoinRewriteUniqueKeyRule TemporalJoinUtil.makeInitialRowTimeTemporalTableJoinCondCall( rexBuilder, diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalGroupWindowAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalGroupWindowAggregateRule.scala index 1ddda73ff20aa..aa9dbbd9e203e 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalGroupWindowAggregateRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalGroupWindowAggregateRule.scala @@ -59,7 +59,6 @@ class StreamPhysicalGroupWindowAggregateRule override def convert(rel: RelNode): RelNode = { val agg = rel.asInstanceOf[FlinkLogicalWindowAggregate] val input = agg.getInput - val inputRowType = input.getRowType val cluster = rel.getCluster val requiredDistribution = if (agg.getGroupCount != 0) { FlinkRelDistribution.hash(agg.getGroupSet.asList) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalIntervalJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalIntervalJoinRule.scala index a2724c425d338..5fe6f80c6a66f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalIntervalJoinRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalIntervalJoinRule.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalI import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.rel.RelNode +import org.apache.flink.table.api.ValidationException import scala.collection.JavaConversions._ @@ -48,6 +49,18 @@ class StreamPhysicalIntervalJoinRule if (windowBounds.isDefined) { if (windowBounds.get.isEventTime) { + val leftTimeAttributeType = join.getLeft.getRowType + .getFieldList + .get(windowBounds.get.getLeftTimeIdx).getType + val rightTimeAttributeType = join.getRight.getRowType + .getFieldList + .get(windowBounds.get.getRightTimeIdx).getType + if (leftTimeAttributeType.getSqlTypeName != rightTimeAttributeType.getSqlTypeName) { + throw new ValidationException( + String.format("Interval join with rowtime attribute requires same rowtime types," + + " but the types are %s and %s.", + leftTimeAttributeType.toString, rightTimeAttributeType.toString)) + } true } else { // Check that no event-time attributes are in the input because the processing time window diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala index 0102f4af80224..b60ae51d794f7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala @@ -246,7 +246,7 @@ class LegacyCatalogSourceTable[T]( TimestampKind.REGULAR, ltz.getPrecision) case _ => throw new ValidationException("The supported time indicator type" + - " are timestamp and timestampLtz, but is " + f.getType + ".") + " are TIMESTAMP and TIMESTAMP_LTZ, but is " + f.getType + ".") } } else { f.getType diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala index 886a7107b0ddb..3742b95037b78 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataType.scala @@ -23,6 +23,8 @@ import org.apache.calcite.sql.`type`.BasicSqlType import java.lang +import org.apache.calcite.sql.`type`.SqlTypeName.TIMESTAMP + /** * Creates a time indicator type for event-time or processing-time, but with similar properties * as a basic SQL type. @@ -45,7 +47,11 @@ class TimeIndicatorRelDataType( } override def toString: String = { - s"TIME ATTRIBUTE(${if (isEventTime) "ROWTIME" else "PROCTIME"})" + // Calcite caches type instance by the type string representation in + // org.apache.calcite.rel.type.RelDataTypeFactoryImpl, thus we use + // unique name for each TimeIndicatorRelDataType + s"${if (typeName == TIMESTAMP) "TIMESTAMP(3)" else "TIMESTAMP_LTZ(3)"}" + + s" ${if (isEventTime) "*ROWTIME*" else "*PROCTIME*"}" } override def generateTypeString(sb: lang.StringBuilder, withDetail: Boolean): Unit = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala index 46c4dd5c356ad..73774cfb2f3b1 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala @@ -184,8 +184,8 @@ object WindowUtil { } val timeAttributeType = FlinkTypeFactory.toLogicalType(fieldType) if (!canBeTimeAttributeType(timeAttributeType)) { - throw new ValidationException("The supported time indicator type are" + - " timestamp and timestampLtz, but is " + FlinkTypeFactory.toLogicalType(fieldType) + "") + throw new ValidationException("The supported time indicator type are TIMESTAMP" + + " and TIMESTAMP_LTZ, but is " + FlinkTypeFactory.toLogicalType(fieldType) + "") } val windowFunction = windowCall.getOperator.asInstanceOf[SqlWindowTableFunction] diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeSerdeTest.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeSerdeTest.java index e2beee4c71b22..ddf0c167660b7 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeSerdeTest.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RelDataTypeSerdeTest.java @@ -188,7 +188,8 @@ public static Collection parameters() { -1)), Arrays.asList("f3", "f4")), FACTORY.createSqlType(SqlTypeName.SARG), - FACTORY.createRowtimeIndicatorType(true), + FACTORY.createRowtimeIndicatorType(true, false), + FACTORY.createRowtimeIndicatorType(true, true), FACTORY.createProctimeIndicatorType(true), FACTORY.createFieldTypeFromLogicalType( new LegacyTypeInformationType<>(LogicalTypeRoot.RAW, Types.STRING)), diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java index ab132892be0e1..6ce26c84042b9 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/GroupWindowAggregateJsonITCase.java @@ -38,7 +38,7 @@ public void setup() throws Exception { super.setup(); createTestValuesSourceTable( "MyTable", - JavaScalaConversionUtil.toJava(TestData.windowData()), + JavaScalaConversionUtil.toJava(TestData.windowDataWithTimestamp()), new String[] { "ts STRING", "`int` INT", diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java index 0e117c8320b9e..e63c322918601 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/TemporalSortJsonITCase.java @@ -56,7 +56,7 @@ public void testSortProcessingTime() throws Exception { public void testSortRowTime() throws Exception { createTestValuesSourceTable( "MyTable", - JavaScalaConversionUtil.toJava(TestData.windowData()), + JavaScalaConversionUtil.toJava(TestData.windowDataWithTimestamp()), new String[] { "ts STRING", "`int` INT", diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java index 938ce6672ee2c..a2bf150ef17dc 100644 --- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java +++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/WindowAggregateJsonITCase.java @@ -38,7 +38,7 @@ public void setup() throws Exception { super.setup(); createTestValuesSourceTable( "MyTable", - JavaScalaConversionUtil.toJava(TestData.windowData()), + JavaScalaConversionUtil.toJava(TestData.windowDataWithTimestamp()), new String[] { "ts STRING", "`int` INT", diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out index 5c0a657b873f1..ebc17b52482fb 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out @@ -89,6 +89,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index bc6649374b942..d76e65ede6041 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 22, + "id" : 150, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 23, + "id" : 151, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 24, + "id" : 152, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -216,6 +217,7 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -227,7 +229,7 @@ } } ], "condition" : null, - "id" : 25, + "id" : 153, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -254,7 +256,7 @@ "description" : "Calc(select=[b, proctime, a])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 26, + "id" : 154, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -328,7 +330,7 @@ }, "namedWindowProperties" : [ ], "needRetraction" : false, - "id" : 27, + "id" : 155, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -363,7 +365,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 28, + "id" : 156, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -383,43 +385,43 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" } ], "edges" : [ { - "source" : 22, - "target" : 23, + "source" : 150, + "target" : 151, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 151, + "target" : 152, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 24, - "target" : 25, + "source" : 152, + "target" : 153, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 25, - "target" : 26, + "source" : 153, + "target" : 154, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 26, - "target" : 27, + "source" : 154, + "target" : 155, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 27, - "target" : 28, + "source" : 155, + "target" : 156, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out index 56e68267e542c..beb54910f5a24 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 35, + "id" : 157, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 36, + "id" : 158, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 37, + "id" : 159, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -216,6 +217,7 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -227,7 +229,7 @@ } } ], "condition" : null, - "id" : 38, + "id" : 160, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -254,7 +256,7 @@ "description" : "Calc(select=[b, proctime, a])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 39, + "id" : 161, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -326,7 +328,7 @@ }, "namedWindowProperties" : [ ], "needRetraction" : false, - "id" : 40, + "id" : 162, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -361,7 +363,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 41, + "id" : 163, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -381,43 +383,43 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" } ], "edges" : [ { - "source" : 35, - "target" : 36, + "source" : 157, + "target" : 158, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 36, - "target" : 37, + "source" : 158, + "target" : 159, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 37, - "target" : 38, + "source" : 159, + "target" : 160, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 38, - "target" : 39, + "source" : 160, + "target" : 161, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 39, - "target" : 40, + "source" : 161, + "target" : 162, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 40, - "target" : 41, + "source" : 162, + "target" : 163, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index faf79d8166244..d42487a72d79b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -40,7 +40,7 @@ } } ] }, - "id" : 8, + "id" : 142, "outputType" : { "type" : "ROW", "nullable" : true, @@ -71,6 +71,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -96,7 +97,7 @@ } } ], "condition" : null, - "id" : 9, + "id" : 143, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -160,7 +161,7 @@ } }, "rowtimeFieldIndex" : 2, - "id" : 10, + "id" : 144, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -204,11 +205,12 @@ "inputIndex" : 1, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 11, + "id" : 145, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -233,7 +235,7 @@ "description" : "Calc(select=[b, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 12, + "id" : 146, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -346,7 +348,7 @@ } } ], "needRetraction" : false, - "id" : 13, + "id" : 147, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -424,7 +426,7 @@ } } ], "condition" : null, - "id" : 14, + "id" : 148, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -468,7 +470,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 15, + "id" : 149, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -495,50 +497,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" } ], "edges" : [ { - "source" : 8, - "target" : 9, + "source" : 142, + "target" : 143, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 9, - "target" : 10, + "source" : 143, + "target" : 144, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 10, - "target" : 11, + "source" : 144, + "target" : 145, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 11, - "target" : 12, + "source" : 145, + "target" : 146, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 12, - "target" : 13, + "source" : 146, + "target" : 147, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 13, - "target" : 14, + "source" : 147, + "target" : 148, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 14, - "target" : 15, + "source" : 148, + "target" : 149, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out index 5581c312d97ab..f892cdea3588d 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out @@ -41,7 +41,7 @@ } } ] }, - "id" : 1, + "id" : 13, "outputType" : { "type" : "ROW", "nullable" : true, @@ -72,6 +72,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -109,7 +110,7 @@ } } ], "condition" : null, - "id" : 2, + "id" : 14, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -173,7 +174,7 @@ } }, "rowtimeFieldIndex" : 2, - "id" : 3, + "id" : 15, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -217,11 +218,12 @@ "inputIndex" : 1, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 4, + "id" : 16, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -246,7 +248,7 @@ "description" : "Calc(select=[a, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 5, + "id" : 17, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -298,7 +300,7 @@ "schema.0.name" : "a" } }, - "id" : 6, + "id" : 18, "outputType" : { "type" : "ROW", "nullable" : true, @@ -346,6 +348,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -383,7 +386,7 @@ } } ], "condition" : null, - "id" : 7, + "id" : 19, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -451,7 +454,7 @@ } }, "rowtimeFieldIndex" : 4, - "id" : 8, + "id" : 20, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -507,11 +510,12 @@ "inputIndex" : 3, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 9, + "id" : 21, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -538,7 +542,7 @@ "description" : "Calc(select=[a, b, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 10, + "id" : 22, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -582,7 +586,7 @@ "rightTimeIndex" : 2 } }, - "id" : 11, + "id" : 23, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -641,7 +645,7 @@ } } ], "condition" : null, - "id" : 12, + "id" : 24, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -677,7 +681,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 13, + "id" : 25, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -697,85 +701,85 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 13, + "target" : 14, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 14, + "target" : 15, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 15, + "target" : 16, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 16, + "target" : 17, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 18, + "target" : 19, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 19, + "target" : 20, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 20, + "target" : 21, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 9, - "target" : 10, + "source" : 21, + "target" : 22, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 11, + "source" : 17, + "target" : 23, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 10, - "target" : 11, + "source" : 22, + "target" : 23, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 11, - "target" : 12, + "source" : 23, + "target" : 24, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 12, - "target" : 13, + "source" : 24, + "target" : 25, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out index f3b5e413eb214..6ff68f1ceb410 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out @@ -76,6 +76,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -347,6 +348,7 @@ "inputIndex" : 3, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], @@ -373,6 +375,7 @@ "inputIndex" : 4, "type" : { "timestampKind" : "ROWTIME", + "typeName" : "TIMESTAMP", "nullable" : true } } ], diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out index d2a45f3731882..60578b86ca5b7 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out @@ -76,6 +76,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -350,6 +351,7 @@ "inputIndex" : 3, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], @@ -376,6 +378,7 @@ "inputIndex" : 4, "type" : { "timestampKind" : "ROWTIME", + "typeName" : "TIMESTAMP", "nullable" : true } } ], diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out index 19205be1e389c..b65d5556d3756 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out @@ -20,7 +20,7 @@ "schema.1.data-type" : "VARCHAR(2147483647)" } }, - "id" : 128, + "id" : 86, "outputType" : { "type" : "ROW", "nullable" : true, @@ -59,11 +59,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 129, + "id" : 87, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -90,7 +91,7 @@ "description" : "Calc(select=[id, name, PROCTIME() AS proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 130, + "id" : 88, "inputProperties" : [ { "requiredDistribution" : { "type" : "SINGLETON" @@ -407,7 +408,7 @@ }, "interval" : null }, - "id" : 131, + "id" : 89, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -448,7 +449,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 132, + "id" : 90, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -470,29 +471,29 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[aid, bid, cid])" } ], "edges" : [ { - "source" : 128, - "target" : 129, + "source" : 86, + "target" : 87, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 129, - "target" : 130, + "source" : 87, + "target" : 88, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 130, - "target" : 131, + "source" : 88, + "target" : 89, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 131, - "target" : 132, + "source" : 89, + "target" : 90, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index b0d9a8474a9ee..2e7215bbc093e 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -47,7 +47,7 @@ } } ] }, - "id" : 98, + "id" : 62, "outputType" : { "type" : "ROW", "nullable" : true, @@ -85,6 +85,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -104,7 +105,7 @@ } } ], "condition" : null, - "id" : 99, + "id" : 63, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -148,7 +149,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 100, + "id" : 64, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -194,6 +195,7 @@ "inputIndex" : 1, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -217,7 +219,7 @@ } } ], "condition" : null, - "id" : 101, + "id" : 65, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -244,7 +246,7 @@ "description" : "Calc(select=[c, proctime, CAST(a) AS $2])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 102, + "id" : 66, "inputProperties" : [ { "requiredDistribution" : { "type" : "SINGLETON" @@ -331,7 +333,7 @@ } ], "originalInputFields" : 3 }, - "id" : 103, + "id" : 67, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -388,7 +390,7 @@ } } ], "condition" : null, - "id" : 104, + "id" : 68, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -425,7 +427,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 105, + "id" : 69, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -445,50 +447,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b])" } ], "edges" : [ { - "source" : 98, - "target" : 99, + "source" : 62, + "target" : 63, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 99, - "target" : 100, + "source" : 63, + "target" : 64, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 100, - "target" : 101, + "source" : 64, + "target" : 65, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 101, - "target" : 102, + "source" : 65, + "target" : 66, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 102, - "target" : 103, + "source" : 66, + "target" : 67, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 103, - "target" : 104, + "source" : 67, + "target" : 68, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 104, - "target" : 105, + "source" : 68, + "target" : 69, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index e8ea6f69e2e9d..65f76032b19d3 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -47,7 +47,7 @@ } } ] }, - "id" : 83, + "id" : 47, "outputType" : { "type" : "ROW", "nullable" : true, @@ -92,6 +92,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -104,7 +105,7 @@ } } ], "condition" : null, - "id" : 84, + "id" : 48, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -148,7 +149,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 85, + "id" : 49, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -201,6 +202,7 @@ "inputIndex" : 2, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -224,7 +226,7 @@ } } ], "condition" : null, - "id" : 86, + "id" : 50, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -253,7 +255,7 @@ "description" : "Calc(select=[a, c, proctime, CAST(a) AS $3])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 87, + "id" : 51, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -359,7 +361,7 @@ } ], "originalInputFields" : 4 }, - "id" : 88, + "id" : 52, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -484,7 +486,7 @@ } } ], "condition" : null, - "id" : 89, + "id" : 53, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -521,7 +523,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 90, + "id" : 54, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -541,50 +543,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b])" } ], "edges" : [ { - "source" : 83, - "target" : 84, + "source" : 47, + "target" : 48, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 84, - "target" : 85, + "source" : 48, + "target" : 49, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 85, - "target" : 86, + "source" : 49, + "target" : 50, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 86, - "target" : 87, + "source" : 50, + "target" : 51, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 87, - "target" : 88, + "source" : 51, + "target" : 52, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 88, - "target" : 89, + "source" : 52, + "target" : 53, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 89, - "target" : 90, + "source" : 53, + "target" : 54, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index f1481e3844641..1204c61d546d4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -47,7 +47,7 @@ } } ] }, - "id" : 91, + "id" : 55, "outputType" : { "type" : "ROW", "nullable" : true, @@ -78,7 +78,7 @@ } }, "rowtimeFieldIndex" : 2, - "id" : 92, + "id" : 56, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -148,11 +148,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 93, + "id" : 57, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -181,7 +182,7 @@ "description" : "Calc(select=[a, c, CAST(a) AS $2, PROCTIME() AS $3])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 94, + "id" : 58, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -299,7 +300,7 @@ } ], "originalInputFields" : 4 }, - "id" : 95, + "id" : 59, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -402,7 +403,7 @@ } } ], "condition" : null, - "id" : 96, + "id" : 60, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -443,7 +444,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 97, + "id" : 61, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -465,43 +466,43 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])" } ], "edges" : [ { - "source" : 91, - "target" : 92, + "source" : 55, + "target" : 56, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 92, - "target" : 93, + "source" : 56, + "target" : 57, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 93, - "target" : 94, + "source" : 57, + "target" : 58, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 94, - "target" : 95, + "source" : 58, + "target" : 59, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 95, - "target" : 96, + "source" : 59, + "target" : 60, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 96, - "target" : 97, + "source" : 60, + "target" : 61, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index f5ccffc3f7a9a..a10c0c1397e05 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -47,7 +47,7 @@ } } ] }, - "id" : 75, + "id" : 39, "outputType" : { "type" : "ROW", "nullable" : true, @@ -92,6 +92,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -104,7 +105,7 @@ } } ], "condition" : null, - "id" : 76, + "id" : 40, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -148,7 +149,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 77, + "id" : 41, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -201,11 +202,12 @@ "inputIndex" : 2, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 78, + "id" : 42, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -232,7 +234,7 @@ "description" : "Calc(select=[a, c, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 79, + "id" : 43, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -316,7 +318,7 @@ "constants" : [ ], "originalInputFields" : 3 }, - "id" : 80, + "id" : 44, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -439,7 +441,7 @@ } } ], "condition" : null, - "id" : 81, + "id" : 45, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -480,7 +482,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 82, + "id" : 46, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -502,50 +504,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])" } ], "edges" : [ { - "source" : 75, - "target" : 76, + "source" : 39, + "target" : 40, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 76, - "target" : 77, + "source" : 40, + "target" : 41, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 77, - "target" : 78, + "source" : 41, + "target" : 42, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 78, - "target" : 79, + "source" : 42, + "target" : 43, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 79, - "target" : 80, + "source" : 43, + "target" : 44, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 80, - "target" : 81, + "source" : 44, + "target" : 45, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 81, - "target" : 82, + "source" : 45, + "target" : 46, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out index c0df93fe3d349..f178986863235 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out @@ -47,7 +47,7 @@ } } ] }, - "id" : 114, + "id" : 78, "outputType" : { "type" : "ROW", "nullable" : true, @@ -92,6 +92,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -104,7 +105,7 @@ } } ], "condition" : null, - "id" : 115, + "id" : 79, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -148,7 +149,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 116, + "id" : 80, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -201,11 +202,12 @@ "inputIndex" : 2, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 117, + "id" : 81, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -232,7 +234,7 @@ "description" : "Calc(select=[a, c, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 118, + "id" : 82, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -332,7 +334,7 @@ } ], "originalInputFields" : 3 }, - "id" : 119, + "id" : 83, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -455,7 +457,7 @@ } } ], "condition" : null, - "id" : 120, + "id" : 84, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -496,7 +498,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 121, + "id" : 85, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -518,50 +520,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])" } ], "edges" : [ { - "source" : 114, - "target" : 115, + "source" : 78, + "target" : 79, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 115, - "target" : 116, + "source" : 79, + "target" : 80, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 116, - "target" : 117, + "source" : 80, + "target" : 81, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 117, - "target" : 118, + "source" : 81, + "target" : 82, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 118, - "target" : 119, + "source" : 82, + "target" : 83, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 119, - "target" : 120, + "source" : 83, + "target" : 84, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 120, - "target" : 121, + "source" : 84, + "target" : 85, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out index f489929258ca6..d7dd3e1da1976 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 106, + "id" : 70, "outputType" : { "type" : "ROW", "nullable" : true, @@ -90,11 +90,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 107, + "id" : 71, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -140,7 +141,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 108, + "id" : 72, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -203,11 +204,12 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 109, + "id" : 73, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -236,7 +238,7 @@ "description" : "Calc(select=[a, b, c, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 110, + "id" : 74, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -386,7 +388,7 @@ } ], "originalInputFields" : 4 }, - "id" : 111, + "id" : 75, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -589,7 +591,7 @@ } } ], "condition" : null, - "id" : 112, + "id" : 76, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -638,7 +640,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 113, + "id" : 77, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -664,50 +666,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c, d, e])" } ], "edges" : [ { - "source" : 106, - "target" : 107, + "source" : 70, + "target" : 71, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 107, - "target" : 108, + "source" : 71, + "target" : 72, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 108, - "target" : 109, + "source" : 72, + "target" : 73, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 109, - "target" : 110, + "source" : 73, + "target" : 74, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 110, - "target" : 111, + "source" : 74, + "target" : 75, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 111, - "target" : 112, + "source" : 75, + "target" : 76, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 112, - "target" : 113, + "source" : 76, + "target" : 77, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out index 168ef7019d121..55f67d6ff0102 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out @@ -40,7 +40,7 @@ } } ] }, - "id" : 55, + "id" : 26, "outputType" : { "type" : "ROW", "nullable" : true, @@ -71,6 +71,7 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } }, { @@ -104,7 +105,7 @@ } } ], "condition" : null, - "id" : 56, + "id" : 27, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -170,7 +171,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 57, + "id" : 28, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -204,7 +205,7 @@ "description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 58, + "id" : 29, "inputProperties" : [ { "requiredDistribution" : { "type" : "SINGLETON" @@ -249,7 +250,7 @@ "nullIsLast" : false } ] }, - "id" : 59, + "id" : 30, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -292,7 +293,7 @@ } } ], "condition" : null, - "id" : 60, + "id" : 31, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -323,7 +324,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 61, + "id" : 32, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -341,43 +342,43 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[a])" } ], "edges" : [ { - "source" : 55, - "target" : 56, + "source" : 26, + "target" : 27, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 56, - "target" : 57, + "source" : 27, + "target" : 28, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 57, - "target" : 58, + "source" : 28, + "target" : 29, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 58, - "target" : 59, + "source" : 29, + "target" : 30, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 59, - "target" : 60, + "source" : 30, + "target" : 31, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 60, - "target" : 61, + "source" : 31, + "target" : 32, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out index 4d06dd7599695..af0990ad2d145 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 43, + "id" : 133, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 44, + "id" : 134, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 45, + "id" : 135, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -231,11 +232,12 @@ "inputIndex" : 3, "type" : { "timestampKind" : "ROWTIME", + "typeName" : "TIMESTAMP", "nullable" : true } } ], "condition" : null, - "id" : 46, + "id" : 136, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -314,7 +316,7 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "id" : 47, + "id" : 137, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -338,7 +340,7 @@ "description" : "LocalWindowAggregate(groupBy=[b], window=[CUMULATE(time_col=[rowtime], max_size=[15 s], step=[5 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 48, + "id" : 138, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -442,7 +444,7 @@ } } } ], - "id" : 49, + "id" : 139, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -527,7 +529,7 @@ } } ], "condition" : null, - "id" : 50, + "id" : 140, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -575,7 +577,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 51, + "id" : 141, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -604,57 +606,57 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2, EXPR$3])" } ], "edges" : [ { - "source" : 43, - "target" : 44, + "source" : 133, + "target" : 134, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 44, - "target" : 45, + "source" : 134, + "target" : 135, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 45, - "target" : 46, + "source" : 135, + "target" : 136, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 46, - "target" : 47, + "source" : 136, + "target" : 137, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 47, - "target" : 48, + "source" : 137, + "target" : 138, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 48, - "target" : 49, + "source" : 138, + "target" : 139, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 49, - "target" : 50, + "source" : 139, + "target" : 140, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 50, - "target" : 51, + "source" : 140, + "target" : 141, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out index 3ff0685121d7f..2f83fcd614355 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 26, + "id" : 116, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 27, + "id" : 117, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 28, + "id" : 118, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -231,11 +232,12 @@ "inputIndex" : 3, "type" : { "timestampKind" : "ROWTIME", + "typeName" : "TIMESTAMP", "nullable" : true } } ], "condition" : null, - "id" : 29, + "id" : 119, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -314,7 +316,7 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "id" : 30, + "id" : 120, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -340,7 +342,7 @@ "description" : "LocalWindowAggregate(groupBy=[b], window=[HOP(time_col=[rowtime], size=[10 s], slide=[5 s])], select=[b, COUNT(c) AS count$0, SUM(a) AS sum$1, COUNT(*) AS count1$2, slice_end('w$) AS $slice_end])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 31, + "id" : 121, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -446,7 +448,7 @@ } } } ], - "id" : 32, + "id" : 122, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -523,7 +525,7 @@ } } ], "condition" : null, - "id" : 33, + "id" : 123, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -562,7 +564,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 34, + "id" : 124, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -584,57 +586,57 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1, EXPR$2])" } ], "edges" : [ { - "source" : 26, - "target" : 27, + "source" : 116, + "target" : 117, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 27, - "target" : 28, + "source" : 117, + "target" : 118, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 28, - "target" : 29, + "source" : 118, + "target" : 119, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 29, - "target" : 30, + "source" : 119, + "target" : 120, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 30, - "target" : 31, + "source" : 120, + "target" : 121, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 31, - "target" : 32, + "source" : 121, + "target" : 122, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 32, - "target" : 33, + "source" : 122, + "target" : 123, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 33, - "target" : 34, + "source" : 123, + "target" : 124, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index 4fc992f5876cc..1e9e609437fdf 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 1, + "id" : 91, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 2, + "id" : 92, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 3, + "id" : 93, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -231,11 +232,12 @@ "inputIndex" : 3, "type" : { "timestampKind" : "ROWTIME", + "typeName" : "TIMESTAMP", "nullable" : true } } ], "condition" : null, - "id" : 4, + "id" : 94, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -349,7 +351,7 @@ "timeAttributeIndex" : 3, "isRowtime" : true }, - "id" : 5, + "id" : 95, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -393,7 +395,7 @@ "description" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, COUNT(distinct$0 c) AS count$2, concat_distinct_agg(c) AS concat_distinct_agg$3, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 6, + "id" : 96, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -552,7 +554,7 @@ } } } ], - "id" : 7, + "id" : 97, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -664,7 +666,7 @@ } } ], "condition" : null, - "id" : 8, + "id" : 98, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -729,7 +731,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 9, + "id" : 99, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -769,57 +771,57 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_start, window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])" } ], "edges" : [ { - "source" : 1, - "target" : 2, + "source" : 91, + "target" : 92, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 2, - "target" : 3, + "source" : 92, + "target" : 93, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 3, - "target" : 4, + "source" : 93, + "target" : 94, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 4, - "target" : 5, + "source" : 94, + "target" : 95, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 5, - "target" : 6, + "source" : 95, + "target" : 96, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 6, - "target" : 7, + "source" : 96, + "target" : 97, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 7, - "target" : 8, + "source" : 97, + "target" : 98, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 8, - "target" : 9, + "source" : 98, + "target" : 99, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out index dbbad550cb017..2bfecd4f25143 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 10, + "id" : 100, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 11, + "id" : 101, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 12, + "id" : 102, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -224,11 +225,12 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 13, + "id" : 103, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -255,7 +257,7 @@ "description" : "Calc(select=[b, c, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 14, + "id" : 104, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -346,7 +348,7 @@ } } } ], - "id" : 15, + "id" : 105, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -396,7 +398,7 @@ } } ], "condition" : null, - "id" : 16, + "id" : 106, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -431,7 +433,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 17, + "id" : 107, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -451,50 +453,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" } ], "edges" : [ { - "source" : 10, - "target" : 11, + "source" : 100, + "target" : 101, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 11, - "target" : 12, + "source" : 101, + "target" : 102, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 12, - "target" : 13, + "source" : 102, + "target" : 103, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 13, - "target" : 14, + "source" : 103, + "target" : 104, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 14, - "target" : 15, + "source" : 104, + "target" : 105, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 15, - "target" : 16, + "source" : 105, + "target" : 106, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 16, - "target" : 17, + "source" : 106, + "target" : 107, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index 3b50691b6c0b8..fc87a32de2fc4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 35, + "id" : 125, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 36, + "id" : 126, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 37, + "id" : 127, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -223,11 +224,12 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 38, + "id" : 128, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -254,7 +256,7 @@ "description" : "Calc(select=[b, a, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 39, + "id" : 129, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -345,7 +347,7 @@ } } } ], - "id" : 40, + "id" : 130, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -395,7 +397,7 @@ } } ], "condition" : null, - "id" : 41, + "id" : 131, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -430,7 +432,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 42, + "id" : 132, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -450,50 +452,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, EXPR$1])" } ], "edges" : [ { - "source" : 35, - "target" : 36, + "source" : 125, + "target" : 126, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 36, - "target" : 37, + "source" : 126, + "target" : 127, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 37, - "target" : 38, + "source" : 127, + "target" : 128, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 38, - "target" : 39, + "source" : 128, + "target" : 129, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 39, - "target" : 40, + "source" : 129, + "target" : 130, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 40, - "target" : 41, + "source" : 130, + "target" : 131, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 41, - "target" : 42, + "source" : 131, + "target" : 132, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index 349032356a947..dfe20e12be7e8 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -27,7 +27,7 @@ "schema.0.name" : "a" } }, - "id" : 18, + "id" : 108, "outputType" : { "type" : "ROW", "nullable" : true, @@ -96,11 +96,12 @@ "operands" : [ ], "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 19, + "id" : 109, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -168,7 +169,7 @@ } }, "rowtimeFieldIndex" : 3, - "id" : 20, + "id" : 110, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -216,11 +217,12 @@ "inputIndex" : 4, "type" : { "timestampKind" : "PROCTIME", + "typeName" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE", "nullable" : false } } ], "condition" : null, - "id" : 21, + "id" : 111, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -245,7 +247,7 @@ "description" : "Calc(select=[b, proctime])" }, { "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange", - "id" : 22, + "id" : 112, "inputProperties" : [ { "requiredDistribution" : { "type" : "HASH", @@ -333,7 +335,7 @@ } } } ], - "id" : 23, + "id" : 113, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -391,7 +393,7 @@ } } ], "condition" : null, - "id" : 24, + "id" : 114, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -435,7 +437,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], - "id" : 25, + "id" : 115, "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -462,50 +464,50 @@ "description" : "Sink(table=[default_catalog.default_database.MySink], fields=[b, window_end, EXPR$2])" } ], "edges" : [ { - "source" : 18, - "target" : 19, + "source" : 108, + "target" : 109, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 19, - "target" : 20, + "source" : 109, + "target" : 110, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 20, - "target" : 21, + "source" : 110, + "target" : 111, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 21, - "target" : 22, + "source" : 111, + "target" : 112, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 22, - "target" : 23, + "source" : 112, + "target" : 113, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 113, + "target" : 114, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 24, - "target" : 25, + "source" : 114, + "target" : 115, "shuffle" : { "type" : "FORWARD" }, diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.xml index a1226752bb406..e06c9e1c7d047 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.xml @@ -36,7 +36,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -80,7 +80,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -125,7 +125,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml new file mode 100644 index 0000000000000..e02a25e065485 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.xml @@ -0,0 +1,66 @@ + + + + + + 0 +) AS T +GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) +]]> + + + (PREV(A.$2, 0), 0)]], inputFields=[[symbol, ts_ltz, price, tax]]) + +- LogicalWatermarkAssigner(rowtime=[ts_ltz], watermark=[-($1, 1000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, Ticker]]) +]]> + + + (PREV(A.$2, 0), 0)}]) + +- Exchange(distribution=[hash[symbol]]) + +- WatermarkAssigner(rowtime=[ts_ltz], watermark=[-(ts_ltz, 1000:INTERVAL SECOND)]) + +- TableSourceScan(table=[[default_catalog, default_database, Ticker]], fields=[symbol, ts_ltz, price, tax]) +]]> + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml index 52cbf71769c79..86dde7e2fb567 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.xml @@ -542,7 +542,7 @@ LogicalAggregate(group=[{0}], EXPR$1=[COUNT($1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{4}]) :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 0:INTERVAL MILLISECOND)]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyDataStream1]]) - +- LogicalTableFunctionScan(invocation=[Rates($cor0.rowtime)], rowType=[RecordType:peek_no_expand(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(PROCTIME) proctime, TIME ATTRIBUTE(ROWTIME) rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($cor0.rowtime)], rowType=[RecordType:peek_no_expand(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml index a96bd4d766adb..fb29ce2f94b47 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/RelTimeIndicatorConverterTest.xml @@ -244,7 +244,7 @@ LogicalProject(rowtime=[$0], proctime=[$3], s=[$4]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml index 93b8d8bc5544b..9d86f0542f5e6 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml @@ -16,50 +16,50 @@ See the License for the specific language governing permissions and limitations under the License. --> - + - + - + - + - + - + + + + + + 10]]> + + + ($1, 10)]) + +- LogicalWatermarkAssigner(rowtime=[d], watermark=[-($3, 5000:INTERVAL SECOND)]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[+($2, 5000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +]]> + + + 10)]) +- TableSourceScan(table=[[default_catalog, default_database, VirtualTable, watermark=[-(+($2, 5000:INTERVAL SECOND), 5000:INTERVAL SECOND)]]], fields=[a, b, c]) ]]> @@ -91,60 +110,60 @@ Calc(select=[c.d.e AS e, c.d AS d]) ]]> - + - + - + - 10]]> + ($1, 10)]) - +- LogicalWatermarkAssigner(rowtime=[d], watermark=[-($3, 5000:INTERVAL SECOND)]) - +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[+($2, 5000:INTERVAL SECOND)]) - +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) +LogicalProject(a=[$0], b=[$1], EXPR$2=[EXTRACT(FLAG(SECOND), $3)]) ++- LogicalWatermarkAssigner(rowtime=[d], watermark=[-($3, 5000:INTERVAL SECOND)]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[+($2, 5000:INTERVAL SECOND)]) + +- LogicalTableScan(table=[[default_catalog, default_database, VirtualTable]]) ]]> 10)]) +Calc(select=[a, b, EXTRACT(FLAG(SECOND), CAST(Reinterpret((c + 5000:INTERVAL SECOND)))) AS EXPR$2]) +- TableSourceScan(table=[[default_catalog, default_database, VirtualTable, watermark=[-(+($2, 5000:INTERVAL SECOND), 5000:INTERVAL SECOND)]]], fields=[a, b, c]) ]]> - + - + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml index ebf65b8d34179..27b0688d26195 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/UnionTest.xml @@ -81,6 +81,41 @@ Union(all=[true], union=[a, b]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) b : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] +- Calc(select=[a, CAST(0.0:DECIMAL(20, 1)) AS EXPR$1]), rowType=[RecordType(INTEGER a, DECIMAL(20, 1) EXPR$1)] +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]), rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)] +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml index baa1dc44b62ea..2ed235835af62 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowRankTest.xml @@ -34,7 +34,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -72,7 +72,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($6), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($6), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -110,7 +110,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -148,7 +148,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($6), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($6), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -185,7 +185,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -222,7 +222,7 @@ WHERE rownum <= 3 LogicalProject(window_start=[$7], window_end=[$8], window_time=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -277,7 +277,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -318,13 +318,13 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -384,7 +384,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -427,11 +427,11 @@ GROUP BY a, window_start, window_end ($2, 1000))], b=[$2], e=[$5], c=[$3]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($0), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIME ATTRIBUTE(ROWTIME) rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($0), 900000:INTERVAL MINUTE)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(rowtime=[$9], a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) +- LogicalFilter(condition=[<=($10, 3)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6], window_start=[$7], window_end=[$8], window_time=[$9], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $7, $8 ORDER BY $1 DESC NULLS LAST)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -487,7 +487,7 @@ LogicalProject(window_start=[$1], window_end=[$2], window_time=[$3], a=[$0], cnt +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt=[$4], sum_d=[$5], max_d=[$6], wAvg=[$7], uv=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $1, $2 ORDER BY $4 DESC NULLS LAST)]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], sum_d=[SUM($4)], max_d=[MAX($4) FILTER $5], wAvg=[weightedAvg($6, $7)], uv=[COUNT(DISTINCT $8)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], window_time=[$9], d=[$3], $f5=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml index 4d0668fc43676..a08f41b7d698d 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/WindowTableFunctionTest.xml @@ -27,7 +27,7 @@ FROM TABLE( ($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -84,7 +84,7 @@ GROUP BY window_start, window_end LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], window_end=[$8], d=[$3], $f3=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalFilter(condition=[>=($7, 2021-01-01 10:10:00)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -123,7 +123,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -168,7 +168,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -221,7 +221,7 @@ GROUP BY window_start, window_end LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(window_start=[$7], window_end=[$8], len=[$10], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + :- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -235,7 +235,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) +- WindowAggregate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], select=[COUNT(*) AS EXPR$2, SUM(len) AS EXPR$3, MAX(d) FILTER $f4 AS EXPR$4, weightedAvg(b, e) AS wAvg, COUNT(DISTINCT c) AS uv, start('w$) AS window_start, end('w$) AS window_end]) +- Exchange(distribution=[single]) +- Calc(select=[window_start, window_end, EXPR$0 AS len, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c]) - +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time, INTEGER EXPR$0)], joinType=[INNER]) + +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) @@ -267,7 +267,7 @@ GROUP BY window_start, window_end LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()], EXPR$3=[SUM($2)], EXPR$4=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(window_start=[$7], window_end=[$8], len=[$10], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) - :- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + :- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -282,7 +282,7 @@ Calc(select=[window_start, window_end, EXPR$2, EXPR$3, EXPR$4, wAvg, uv]) +- Exchange(distribution=[single]) +- LocalWindowAggregate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], select=[COUNT(*) AS count1$0, SUM(len) AS sum$1, MAX(d) FILTER $f4 AS max$2, weightedAvg(b, e) AS weightedavg$3, COUNT(distinct$0 c) AS count$4, DISTINCT(c) AS distinct$0, slice_end('w$) AS $window_end]) +- Calc(select=[window_start, window_end, EXPR$0 AS len, d, IS TRUE(>(b, 1000)) AS $f4, b, e, c]) - +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time, INTEGER EXPR$0)], joinType=[INNER]) + +- Correlate(invocation=[len_udtf($cor0.c)], correlate=[table(len_udtf($cor0.c))], select=[a,b,c,d,e,rowtime,proctime,window_start,window_end,window_time,EXPR$0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time, INTEGER EXPR$0)], joinType=[INNER]) +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])]) +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)]) +- Calc(select=[a, b, c, d, e, rowtime, PROCTIME() AS proctime]) @@ -306,7 +306,7 @@ GROUP BY CUBE (a, b, window_start, window_end) LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 1, 2}, {0, 1, 3}, {0, 1}, {0, 2, 3}, {0, 2}, {0, 3}, {0}, {1, 2, 3}, {1, 2}, {1, 3}, {1}, {2, 3}, {2}, {3}, {}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -342,7 +342,7 @@ GROUP BY GROUPING SETS ((a, window_start), (window_start)) LogicalProject(a=[$0], uv=[$2]) +- LogicalAggregate(group=[{0, 1}], groups=[[{0, 1}, {1}]], uv=[COUNT(DISTINCT $2)]) +- LogicalProject(a=[$0], window_start=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -378,7 +378,7 @@ GROUP BY GROUPING SETS ((a), (window_start), (window_end)) LogicalProject(a=[$0], uv=[$3]) +- LogicalAggregate(group=[{0, 1, 2}], groups=[[{0}, {1}, {2}]], uv=[COUNT(DISTINCT $3)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -417,7 +417,7 @@ GROUP BY window_start ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -451,7 +451,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -646,7 +646,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -689,7 +689,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 1500000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 1500000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -726,7 +726,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 1500000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 1500000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -765,7 +765,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -807,7 +807,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -854,7 +854,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($6), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($6), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -894,7 +894,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -934,7 +934,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -971,7 +971,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1008,7 +1008,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($6, DESCRIPTOR($5), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1046,7 +1046,7 @@ GROUP BY CUBE (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {1, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1083,7 +1083,7 @@ GROUP BY CUBE (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {1, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1124,7 +1124,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1168,7 +1168,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1211,7 +1211,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1248,7 +1248,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1287,7 +1287,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1329,7 +1329,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], uv=[$5]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1376,7 +1376,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($6), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($6), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1416,7 +1416,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1456,7 +1456,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1493,7 +1493,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1530,7 +1530,7 @@ GROUP BY ROLLUP (a, b), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2, 3}, {0, 2, 3}, {2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($6, DESCRIPTOR($5), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1557,7 +1557,7 @@ Calc(select=[a, b, uv]) LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, window_end, wAvg]) +- LogicalAggregate(group=[{0, 1}], wAvg=[weightedAvg($2, $3)]) +- LogicalProject(window_start=[$7], window_end=[$8], b=[$1], e=[$4]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1566,7 +1566,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, window_end, EXPR$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) +- LogicalProject(window_start=[$7], window_end=[$8]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1601,7 +1601,7 @@ Sink(table=[default_catalog.default_database.s1], fields=[window_start, window_e LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, window_end, wAvg]) +- LogicalAggregate(group=[{0, 1}], wAvg=[weightedAvg($2, $3)]) +- LogicalProject(window_start=[$7], window_end=[$8], b=[$1], e=[$4]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1610,7 +1610,7 @@ LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, w LogicalSink(table=[default_catalog.default_database.s1], fields=[window_start, window_end, EXPR$2]) +- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()]) +- LogicalProject(window_start=[$7], window_end=[$8]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1666,7 +1666,7 @@ GROUP BY a, window_start, window_end LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalFilter(condition=[>($1, 1000)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1710,7 +1710,7 @@ GROUP BY a, window_start, window_end LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)], EXPR$5=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) +- LogicalFilter(condition=[>($1, 1000)]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1748,11 +1748,11 @@ GROUP BY a, window_start, window_end ($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1881,7 +1881,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], window_time=[$9]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1927,7 +1927,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -1970,7 +1970,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2015,7 +2015,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2050,7 +2050,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2086,7 +2086,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2123,7 +2123,7 @@ GROUP BY GROUPING SETS ((a, window_start, window_end), (b, window_start, window_ LogicalProject(a=[$0], b=[$3], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2159,7 +2159,7 @@ GROUP BY GROUPING SETS ((a, window_start, window_end), (b, window_start, window_ LogicalProject(a=[$0], b=[$3], uv=[$4]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 1, 2}, {1, 2, 3}]], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2199,7 +2199,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[SUM($4)], EXPR$4=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2244,7 +2244,7 @@ GROUP BY GROUPING SETS ((a), (b)), window_start, window_end LogicalProject(a=[$0], b=[$1], EXPR$2=[$4], EXPR$3=[$5], EXPR$4=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], groups=[[{0, 2, 3}, {1, 2, 3}]], EXPR$2=[COUNT()], EXPR$3=[SUM($4)], EXPR$4=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(a=[$0], b=[$1], window_start=[$7], window_end=[$8], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2293,7 +2293,7 @@ GROUP BY window_start, a, window_end, b LogicalProject(a=[$1], b=[$3], window_start=[$0], window_end=[$2], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], a=[$0], window_end=[$8], b=[$1], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2333,7 +2333,7 @@ GROUP BY window_start, a, window_end, b LogicalProject(a=[$1], b=[$3], window_start=[$0], window_end=[$2], EXPR$4=[$4], EXPR$5=[$5], EXPR$6=[$6], uv=[$7]) +- LogicalAggregate(group=[{0, 1, 2, 3}], EXPR$4=[COUNT()], EXPR$5=[SUM($4)], EXPR$6=[MAX($4) FILTER $5], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], a=[$0], window_end=[$8], b=[$1], d=[$3], $f5=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2371,7 +2371,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2416,7 +2416,7 @@ GROUP BY a, window_start, window_end, ws, window_time LogicalProject(a=[$0], window_start=[$1], ws=[$3], window_end=[$2], window_time=[$4], EXPR$5=[$5], EXPR$6=[$6], EXPR$7=[$7], wAvg=[$8], uv=[$9]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()], EXPR$6=[SUM($5)], EXPR$7=[MAX($5) FILTER $6], wAvg=[weightedAvg($7, $8)], uv=[COUNT(DISTINCT $9)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], ws=[$7], window_time=[$9], d=[$3], $f6=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2461,7 +2461,7 @@ GROUP BY a, window_start, window_end, ws, window_time LogicalProject(a=[$0], window_start=[$1], ws=[$3], window_end=[$2], window_time=[$4], EXPR$5=[$5], EXPR$6=[$6], EXPR$7=[$7], wAvg=[$8], uv=[$9]) +- LogicalAggregate(group=[{0, 1, 2, 3, 4}], EXPR$5=[COUNT()], EXPR$6=[SUM($5)], EXPR$7=[MAX($5) FILTER $6], wAvg=[weightedAvg($7, $8)], uv=[COUNT(DISTINCT $9)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], ws=[$7], window_time=[$9], d=[$3], $f6=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2500,7 +2500,7 @@ GROUP BY 'literal', window_start, window_end LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)]) +- LogicalProject($f0=[_UTF-16LE'literal'], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2538,7 +2538,7 @@ GROUP BY 'literal', window_start, window_end LogicalProject(window_start=[$1], window_end=[$2], EXPR$2=[$3], EXPR$3=[$4], EXPR$4=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$2=[COUNT()], EXPR$3=[SUM($3)], EXPR$4=[MAX($3) FILTER $4], uv=[COUNT(DISTINCT $5)]) +- LogicalProject($f0=[_UTF-16LE'literal'], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2576,7 +2576,7 @@ GROUP BY window_start, window_end ($1, 1000))], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2614,7 +2614,7 @@ GROUP BY window_start, window_end LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], wAvg=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT()], EXPR$1=[SUM($2)], EXPR$2=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], window_end=[$8], d=[$3], $f3=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2651,7 +2651,7 @@ GROUP BY window_start, window_end LogicalProject(EXPR$0=[$2], EXPR$1=[$3], EXPR$2=[$4], wAvg=[$5], uv=[$6]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[COUNT()], EXPR$1=[SUM($2)], EXPR$2=[MAX($2) FILTER $3], wAvg=[weightedAvg($4, $5)], uv=[COUNT(DISTINCT $6)]) +- LogicalProject(window_start=[$7], window_end=[$8], d=[$3], $f3=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2691,7 +2691,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($6), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2730,7 +2730,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2769,7 +2769,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2806,7 +2806,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -2994,7 +2994,7 @@ GROUP BY a, window_start, window_end ($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) @@ -3034,7 +3034,7 @@ GROUP BY a, window_start, window_end LogicalProject(a=[$0], EXPR$1=[$3], EXPR$2=[$4], EXPR$3=[$5], wAvg=[$6], uv=[$7], window_start=[$1], window_end=[$2]) +- LogicalAggregate(group=[{0, 1, 2}], EXPR$1=[COUNT()], EXPR$2=[SUM($3)], EXPR$3=[MAX($3) FILTER $4], wAvg=[weightedAvg($5, $6)], uv=[COUNT(DISTINCT $7)]) +- LogicalProject(a=[$0], window_start=[$7], window_end=[$8], d=[$3], $f4=[IS TRUE(>($1, 1000))], b=[$1], e=[$4], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($6, DESCRIPTOR($5), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, DECIMAL(10, 3) d, BIGINT e, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[$6]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($5, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], rowtime=[$5], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.xml index 771cc545bf164..24f1cec851a9b 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.xml @@ -16,32 +16,75 @@ See the License for the specific language governing permissions and limitations under the License. --> - + =($4, -($9, 600000:INTERVAL MINUTE)), <=($4, +($9, 3600000:INTERVAL HOUR)))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) ]]> = (CAST(rowtime0) - 600000:INTERVAL MINUTE)) AND (CAST(rowtime) <= (CAST(rowtime0) + 3600000:INTERVAL HOUR)))], select=[a, b, rowtime, a0, b0, rowtime0]) + :- Exchange(distribution=[hash[a]]) + : +- Calc(select=[a, b, rowtime]) + : +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) + +- Exchange(distribution=[hash[a]]) + +- Calc(select=[a, b, rowtime]) + +- DataStreamScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, proctime, rowtime]) +]]> + + + + + t2.rowtime - INTERVAL '5' SECOND AND + t1.rowtime < t2.rowtime + INTERVAL '5' SECOND + ]]> + + + ($3, -($7, 5000:INTERVAL SECOND)), <($3, +($7, 5000:INTERVAL SECOND)))], joinType=[inner]) + :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$3]) + : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[TO_TIMESTAMP_LTZ($1, 3)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable3]]) + +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$3]) + +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[TO_TIMESTAMP_LTZ($1, 3)]) + +- LogicalTableScan(table=[[default_catalog, default_database, MyTable4]]) +]]> + + + (rowtime0 - 5000:INTERVAL SECOND)) AND (rowtime < (rowtime0 + 5000:INTERVAL SECOND)))], select=[a, rowtime, a0, rowtime0]) + :- Exchange(distribution=[hash[a]]) + : +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + : +- Calc(select=[a, TO_TIMESTAMP_LTZ(b, 3) AS rowtime]) + : +- TableSourceScan(table=[[default_catalog, default_database, MyTable3, project=[a, b]]], fields=[a, b]) + +- Exchange(distribution=[hash[a]]) + +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) + +- Calc(select=[a, TO_TIMESTAMP_LTZ(b, 3) AS rowtime]) + +- TableSourceScan(table=[[default_catalog, default_database, MyTable4, project=[a, b]]], fields=[a, b]) ]]> @@ -219,42 +262,6 @@ Calc(select=[a, b]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, proctime, rowtime]) -]]> - - - - - - - - =($4, -($9, 600000:INTERVAL MINUTE)), <=($4, +($9, 3600000:INTERVAL HOUR)))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) - +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) -]]> - - - = (CAST(rowtime0) - 600000:INTERVAL MINUTE)) AND (CAST(rowtime) <= (CAST(rowtime0) + 3600000:INTERVAL HOUR)))], select=[a, b, rowtime, a0, b0, rowtime0]) - :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, rowtime]) - : +- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, proctime, rowtime]) - +- Exchange(distribution=[hash[a]]) - +- Calc(select=[a, b, rowtime]) - +- DataStreamScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, proctime, rowtime]) ]]> @@ -285,6 +292,35 @@ Calc(select=[a, b]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, b, proctime]) +- DataStreamScan(table=[[default_catalog, default_database, MyTable2]], fields=[a, b, c, proctime, rowtime]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.xml index 01c75b98d74db..744dd5570b230 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalFunctionJoinTest.xml @@ -26,7 +26,7 @@ LogicalProject(rate=[*($0, $4)]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(ROWTIME) rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -53,7 +53,7 @@ LogicalProject(rate=[$0], secondary_key=[$1], t3_comment=[$2], t3_secondary_key= : +- LogicalFilter(condition=[AND(=($7, $3), OR(>($8, 120), =($9, $4)))]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{0}]) : :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - : +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(TIME ATTRIBUTE(ROWTIME) rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)]) + : +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)]) +- LogicalTableScan(table=[[default_catalog, default_database, Table3]]) ]]> @@ -86,7 +86,7 @@ LogicalProject(rate=[*($0, $4)]) :- LogicalProject(o_amount=[$0], o_currency=[$1], o_rowtime=[$2]) : +- LogicalFilter(condition=[>($0, 1000)]) : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(ROWTIME) rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($cor0.o_rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -111,7 +111,7 @@ LogicalProject(rate=[*($0, $4)]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}]) :- LogicalTableScan(table=[[default_catalog, default_database, ProctimeOrders]]) - +- LogicalTableFunctionScan(invocation=[ProctimeRates($cor0.o_proctime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(PROCTIME) proctime)]) + +- LogicalTableFunctionScan(invocation=[ProctimeRates($cor0.o_proctime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml index 138f754d4e4b6..21e73d9f47e0c 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.xml @@ -112,6 +112,39 @@ Calc(select=[amount, currency, rowtime, PROCTIME_MATERIALIZE(proctime) AS procti +- Exchange(distribution=[hash[currency]]) +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, RatesHistoryLegacy, source: [CollectionTableSource(currency, rate, rowtime)]]], fields=[currency, rate, rowtime]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml index 6d3bec82f8539..abe04d55abffb 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/WindowJoinTest.xml @@ -38,13 +38,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -92,13 +92,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -146,13 +146,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -200,13 +200,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -252,13 +252,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -304,13 +304,13 @@ ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$8], b0=[$9], c0=[$10]) +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -375,14 +375,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -431,14 +431,14 @@ LogicalProject(rowtime=[$0], a=[$1], l_cnt=[$2], l_uv=[$3], r_cnt=[$4], r_uv=[$5 : +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) : :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -519,14 +519,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -593,14 +593,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($3), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -668,14 +668,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[HOP($4, DESCRIPTOR($4), 300000:INTERVAL MINUTE, 600000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -740,14 +740,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -813,14 +813,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -867,13 +867,13 @@ LogicalProject(rowtime=[$0], a=[$1], l_b=[$2], l_c=[$3], r_b=[$4], r_c=[$5], a0= :- LogicalProject(rowtime=[$7], a=[$0], l_b=[$1], l_c=[$2], r_b=[$9], r_c=[$10]) : +- LogicalJoin(condition=[AND(=($5, $13), =($6, $14), =($0, $8))], joinType=[inner]) : :- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4], window_start=[$5], window_end=[$6], window_time=[$7]) - : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[TUMBLE($4, DESCRIPTOR($3), 900000:INTERVAL MINUTE)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -949,14 +949,14 @@ LogicalProject(a=[$0], window_start=[$1], window_end=[$2], window_time=[$3], cnt +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6), >(CAST($1):BIGINT NOT NULL, $11))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(PROCTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($4), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP_LTZ(3) *PROCTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) @@ -1007,14 +1007,14 @@ LogicalProject(window_start=[$0], window_end=[$1], a=[$2], l_cnt=[$3], l_uv=[$4] +- LogicalJoin(condition=[AND(=($1, $7), =($2, $8), =($0, $6))], joinType=[inner]) :- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) : +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + : +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) : +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) : +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]]) +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)]) +- LogicalProject(a=[$0], window_start=[$5], window_end=[$6], window_time=[$7], c=[$2]) - +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIME ATTRIBUTE(ROWTIME) rowtime, TIME ATTRIBUTE(PROCTIME) proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIME ATTRIBUTE(ROWTIME) window_time)]) + +- LogicalTableFunctionScan(invocation=[CUMULATE($4, DESCRIPTOR($3), 600000:INTERVAL MINUTE, 3600000:INTERVAL HOUR)], rowType=[RecordType(INTEGER a, VARCHAR(2147483647) b, BIGINT c, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[$4]) +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($3, 1000:INTERVAL SECOND)]) +- LogicalProject(a=[$0], b=[$1], c=[$2], rowtime=[$3], proctime=[PROCTIME()]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml index 52b9100b00c17..cdffb5857550c 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml @@ -25,7 +25,7 @@ LogicalJoin(condition=[=($3, $1)], joinType=[inner]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) : :- LogicalProject(o_rowtime=[AS($0, _UTF-16LE'o_rowtime')], o_comment=[AS($1, _UTF-16LE'o_comment')], o_amount=[AS($2, _UTF-16LE'o_amount')], o_currency=[AS($3, _UTF-16LE'o_currency')], o_secondary_key=[AS($4, _UTF-16LE'o_secondary_key')]) : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) -: +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$879e2e896200363f113e410b6aadda05($0)], rowType=[RecordType(TIME ATTRIBUTE(ROWTIME) rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)], elementType=[class [Ljava.lang.Object;]) +: +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$879e2e896200363f113e410b6aadda05($0)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)], elementType=[class [Ljava.lang.Object;]) +- LogicalTableScan(table=[[default_catalog, default_database, ThirdTable]]) ]]> @@ -53,7 +53,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$7b64fb5334c47de8df24f3ab20394c19($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(ROWTIME) rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$7b64fb5334c47de8df24f3ab20394c19($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) ]]> @@ -74,7 +74,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(ROWTIME) rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -95,7 +95,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, ProctimeOrders]]) - +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$3f15d21732a0cca300e47b78077046df($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(PROCTIME) proctime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$3f15d21732a0cca300e47b78077046df($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)], elementType=[class [Ljava.lang.Object;]) ]]> @@ -117,7 +117,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$7b64fb5334c47de8df24f3ab20394c19($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIME ATTRIBUTE(ROWTIME) rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[org$apache$flink$table$functions$TemporalTableFunctionImpl$7b64fb5334c47de8df24f3ab20394c19($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataTypeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataTypeTest.scala index d3a882b19a774..600ddfe955034 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataTypeTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/schema/TimeIndicatorRelDataTypeTest.scala @@ -32,11 +32,14 @@ class TimeIndicatorRelDataTypeTest { def testGenerateTypeString() { val typeFactory = new FlinkTypeFactory(new FlinkTypeSystem) assertEquals( - "TIME ATTRIBUTE(PROCTIME) NOT NULL", + "TIMESTAMP_LTZ(3) *PROCTIME* NOT NULL", typeFactory.createProctimeIndicatorType(false).getFullTypeString) assertEquals( - "TIME ATTRIBUTE(ROWTIME) NOT NULL", - typeFactory.createRowtimeIndicatorType(false).getFullTypeString) + "TIMESTAMP(3) *ROWTIME* NOT NULL", + typeFactory.createRowtimeIndicatorType(false, false).getFullTypeString) + assertEquals( + "TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL", + typeFactory.createRowtimeIndicatorType(false, true).getFullTypeString) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala new file mode 100644 index 0000000000000..248af5b409f64 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MatchRecognizeTest.scala @@ -0,0 +1,71 @@ +/* + * 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.planner.plan.stream.sql + +import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase} +import org.junit.{Before, Test} + +class MatchRecognizeTest extends TableTestBase { + + protected val util: StreamTableTestUtil = streamTestUtil() + + @Before + def before(): Unit = { + val ddl = + """ + |CREATE TABLE Ticker ( + | `symbol` STRING, + | `ts_ltz` TIMESTAMP_LTZ(3), + | `price` INT, + | `tax` INT, + | WATERMARK FOR `ts_ltz` AS `ts_ltz` - INTERVAL '1' SECOND + |) WITH ( + | 'connector' = 'values' + |) + |""".stripMargin + util.tableEnv.executeSql(ddl) + } + + @Test + def testMatchRecognizeOnRowtimeLTZ(): Unit = { + val sqlQuery = + s""" + |SELECT + | symbol, + | SUM(price) as price, + | TUMBLE_ROWTIME(matchRowtime, interval '3' second) as rowTime, + | TUMBLE_START(matchRowtime, interval '3' second) as startTime + |FROM Ticker + |MATCH_RECOGNIZE ( + | PARTITION BY symbol + | ORDER BY ts_ltz + | MEASURES + | A.price as price, + | A.tax as tax, + | MATCH_ROWTIME() as matchRowtime + | ONE ROW PER MATCH + | PATTERN (A) + | DEFINE + | A AS A.price > 0 + |) AS T + |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) + |""".stripMargin + util.verifyRelPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala index cc0c24cbafcb5..3c8994356e32e 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.scala @@ -32,45 +32,43 @@ class SourceWatermarkTest extends TableTestBase { @Before def setup(): Unit = { - val ddl1 = - """ - | CREATE TABLE VirtualTable ( - | a INT, - | b BIGINT, - | c TIMESTAMP(3), - | d AS c + INTERVAL '5' SECOND, - | WATERMARK FOR d AS d - INTERVAL '5' SECOND - | ) WITH ( - | 'connector' = 'values', - | 'enable-watermark-push-down' = 'true', - | 'bounded' = 'false', - | 'disable-lookup' = 'true' - | ) - |""".stripMargin - util.tableEnv.executeSql(ddl1) - - val ddl2 = - """ - | CREATE TABLE NestedTable ( - | a INT, - | b BIGINT, - | c ROW>, - | g AS c.d.f, - | WATERMARK FOR g AS g - INTERVAL '5' SECOND - | ) WITH ( - | 'connector' = 'values', - | 'enable-watermark-push-down' = 'true', - | 'nested-projection-supported' = 'true', - | 'bounded' = 'false', - | 'disable-lookup' = 'true' - | ) - |""".stripMargin - util.tableEnv.executeSql(ddl2) + util.tableEnv.executeSql( + s""" + | CREATE TABLE VirtualTable ( + | a INT, + | b BIGINT, + | c TIMESTAMP(3), + | d AS c + INTERVAL '5' SECOND, + | WATERMARK FOR d AS d - INTERVAL '5' SECOND + | ) WITH ( + | 'connector' = 'values', + | 'enable-watermark-push-down' = 'true', + | 'bounded' = 'false', + | 'disable-lookup' = 'true' + | ) + """.stripMargin) + + util.tableEnv.executeSql( + s""" + | CREATE TABLE NestedTable ( + | a INT, + | b BIGINT, + | c ROW>, + | g AS c.d.f, + | WATERMARK FOR g AS g - INTERVAL '5' SECOND + | ) WITH ( + | 'connector' = 'values', + | 'enable-watermark-push-down' = 'true', + | 'nested-projection-supported' = 'true', + | 'bounded' = 'false', + | 'disable-lookup' = 'true' + | ) + """.stripMargin) JavaFunc5.closeCalled = false JavaFunc5.openCalled = false util.tableEnv.createTemporarySystemFunction("func", new JavaFunc5) - val ddl3 = + util.tableEnv.executeSql( s""" | CREATE Table UdfTable ( | a INT, @@ -84,28 +82,43 @@ class SourceWatermarkTest extends TableTestBase { | 'enable-watermark-push-down' = 'true', | 'disable-lookup' = 'true' | ) - |""".stripMargin - util.tableEnv.executeSql(ddl3) - - val ddl4 = - """ - | CREATE TABLE MyTable( - | a INT, - | b BIGINT, - | c TIMESTAMP(3), - | originTime BIGINT METADATA, - | rowtime AS TO_TIMESTAMP(FROM_UNIXTIME(originTime/1000), 'yyyy-MM-dd HH:mm:ss'), - | WATERMARK FOR rowtime AS rowtime - | ) WITH ( - | 'connector' = 'values', - | 'enable-watermark-push-down' = 'true', - | 'bounded' = 'false', - | 'disable-lookup' = 'true', - | 'readable-metadata' = 'originTime:BIGINT' - | ) - |""".stripMargin - - util.tableEnv.executeSql(ddl4) + """.stripMargin) + + util.tableEnv.executeSql( + s""" + | CREATE TABLE MyTable( + | a INT, + | b BIGINT, + | c TIMESTAMP(3), + | originTime BIGINT METADATA, + | rowtime AS TO_TIMESTAMP(FROM_UNIXTIME(originTime/1000), 'yyyy-MM-dd HH:mm:ss'), + | WATERMARK FOR rowtime AS rowtime + | ) WITH ( + | 'connector' = 'values', + | 'enable-watermark-push-down' = 'true', + | 'bounded' = 'false', + | 'disable-lookup' = 'true', + | 'readable-metadata' = 'originTime:BIGINT' + | ) + """.stripMargin) + + util.tableEnv.executeSql( + s""" + | CREATE TABLE MyLtzTable( + | a INT, + | b BIGINT, + | c TIMESTAMP(3), + | originTime BIGINT METADATA, + | rowtime AS TO_TIMESTAMP_LTZ(originTime, 3), + | WATERMARK FOR rowtime AS rowtime + | ) WITH ( + | 'connector' = 'values', + | 'enable-watermark-push-down' = 'true', + | 'bounded' = 'false', + | 'disable-lookup' = 'true', + | 'readable-metadata' = 'originTime:BIGINT' + | ) + """.stripMargin) } @Test @@ -138,6 +151,11 @@ class SourceWatermarkTest extends TableTestBase { util.verifyExecPlan("SELECT a, b FROM MyTable") } + @Test + def testWatermarkOnTimestampLtzCol(): Unit = { + util.verifyExecPlan("SELECT a, b FROM MyLtzTable") + } + @Test def testProjectTransposeWatermarkAssigner(): Unit = { val sourceDDL = diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala index 894a9030d21d8..0d4ff1a99708c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/UnionTest.scala @@ -27,13 +27,60 @@ import org.junit.{Before, Test} // TODO add more union case after aggregation and join supported class UnionTest extends TableTestBase { - private val util = batchTestUtil() + private val util = streamTestUtil() @Before def before(): Unit = { util.addTableSource[(Int, Long, String)]("MyTable1", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c) util.addTableSource[(Int, Long, String)]("MyTable3", 'a, 'b, 'c) + + util.tableEnv.executeSql( + s""" + |CREATE TABLE t1 ( + | id int, + | ts bigint, + | name string, + | timestamp_col timestamp(3), + | val bigint, + | name varchar(32), + | timestamp_ltz_col as TO_TIMESTAMP_LTZ(ts, 3), + | watermark for timestamp_col as timestamp_col + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false' + |) + """.stripMargin) + + util.tableEnv.executeSql( + s""" + |CREATE TABLE t2 ( + | id int, + | ts bigint, + | name string, + | timestamp_col timestamp(3), + | timestamp_ltz_col as TO_TIMESTAMP_LTZ(ts, 3), + | watermark for timestamp_ltz_col as timestamp_ltz_col + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false' + |) + """.stripMargin) + + util.tableEnv.executeSql( + s""" + |CREATE TABLE t3 ( + | id int, + | ts bigint, + | name string, + | timestamp_col timestamp(3), + | timestamp_ltz_col as TO_TIMESTAMP_LTZ(ts, 3), + | watermark for timestamp_ltz_col as timestamp_ltz_col + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false' + |) + """.stripMargin) } @Test @@ -63,4 +110,33 @@ class UnionTest extends TableTestBase { util.verifyRelPlanWithType(sqlQuery) } + @Test + def testUnionDiffRowTime(): Unit = { + expectedException.expectMessage( + "Union fields with time attributes requires same types," + + " but the types are TIMESTAMP_LTZ(3) *ROWTIME* and TIMESTAMP(3) *ROWTIME*.") + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT id, ts, name, timestamp_col FROM t1 + | UNION ALL + | SELECT id, ts, name, timestamp_ltz_col FROM t2) + """.stripMargin + + util.verifyRelPlanWithType(sqlQuery) + } + + @Test + def testUnionSameRowTime(): Unit = { + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT id, ts, name, timestamp_col, timestamp_ltz_col FROM t2 + | UNION ALL + | SELECT id, ts, name, timestamp_col, timestamp_ltz_col FROM t3) + """.stripMargin + + util.verifyRelPlanWithType(sqlQuery) + } + } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala index 5076a4377889f..fac2a86edc7d4 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/IntervalJoinTest.scala @@ -36,6 +36,33 @@ class IntervalJoinTest extends TableTestBase { util.addDataStream[(Int, String, Long)]( "MyTable2", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime) + util.tableEnv.executeSql( + s""" + |CREATE TABLE MyTable3 ( + | a int, + | b bigint, + | c string, + | rowtime as TO_TIMESTAMP_LTZ(b, 3), + | watermark for rowtime as rowtime + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false' + |) + """.stripMargin) + util.tableEnv.executeSql( + s""" + |CREATE TABLE MyTable4 ( + | a int, + | b bigint, + | c string, + | rowtime as TO_TIMESTAMP_LTZ(b, 3), + | watermark for rowtime as rowtime + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false' + |) + """.stripMargin) + /** There should exist exactly two time conditions **/ @Test(expected = classOf[TableException]) def testInteravlJoinSingleTimeCondition(): Unit = { @@ -60,6 +87,23 @@ class IntervalJoinTest extends TableTestBase { util.verifyExecPlan(sql) } + + /** Both rowtime types in a join condition must be of the same type **/ + @Test + def testIntervalJoinOnDiffRowTimeType(): Unit = { + expectedException.expectMessage( + "Interval join with rowtime attribute requires same rowtime types," + + " but the types are TIMESTAMP(3) *ROWTIME* and TIMESTAMP_LTZ(3) *ROWTIME*") + val sql = + """ + |SELECT t2.a FROM MyTable2 t1 JOIN MyTable3 t2 ON + | t1.a = t2.a AND + | t1.rowtime > t2.rowtime - INTERVAL '5' SECOND AND + | t1.rowtime < t2.rowtime + INTERVAL '5' SECOND + """.stripMargin + util.verifyExecPlan(sql) + } + /** The time conditions should be an And condition **/ @Test(expected = classOf[TableException]) def testInteravalNotCnfCondition(): Unit = { @@ -149,6 +193,18 @@ class IntervalJoinTest extends TableTestBase { util.verifyExecPlan(sqlQuery) } + @Test + def testIntervalJoinOnTimestampLtzRowtime(): Unit = { + val sql = + """ + |SELECT t2.a FROM MyTable3 t1 JOIN MyTable4 t2 ON + | t1.a = t2.a AND + | t1.rowtime > t2.rowtime - INTERVAL '5' SECOND AND + | t1.rowtime < t2.rowtime + INTERVAL '5' SECOND + """.stripMargin + util.verifyExecPlan(sql) + } + @Test def testRowTimeInnerJoinWithoutEqualCondition(): Unit = { val sqlQuery = diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala index 8901fc88025d5..2750ac1a3c24c 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/TemporalJoinTest.scala @@ -147,6 +147,32 @@ class TemporalJoinTest extends TableTestBase { util.addTable("CREATE VIEW rates_last_value AS SELECT currency, LAST_VALUE(rate) AS rate " + "FROM RatesHistory " + "GROUP BY currency ") + + util.tableEnv.executeSql( + s""" + |CREATE TABLE OrdersLtz ( + | amount INT, + | currency STRING, + | ts BIGINT, + | rowtime AS TO_TIMESTAMP_LTZ(ts, 3), + | WATERMARK FOR rowtime AS rowtime + |) WITH ( + | 'connector' = 'values' + |) + """.stripMargin) + util.tableEnv.executeSql( + s""" + |CREATE TABLE RatesLtz ( + | currency STRING, + | rate INT, + | ts BIGINT, + | rowtime as TO_TIMESTAMP_LTZ(ts, 3), + | WATERMARK FOR rowtime AS rowtime, + | PRIMARY KEY(currency) NOT ENFORCED + |) WITH ( + | 'connector' = 'values' + |) + """.stripMargin) } @Test @@ -179,6 +205,16 @@ class TemporalJoinTest extends TableTestBase { util.verifyExecPlan(sqlQuery) } + + @Test + def testEventTimeTemporalJoinOnTimestampLtzRowtime(): Unit = { + val sqlQuery = "SELECT * " + + "FROM OrdersLtz AS o JOIN " + + "RatesLtz FOR SYSTEM_TIME AS OF o.rowtime AS r " + + "ON o.currency = r.currency" + util.verifyExecPlan(sqlQuery) + } + @Test def testEventTimeTemporalJoinWithView(): Unit = { val sqlQuery = "SELECT * " + @@ -468,6 +504,19 @@ class TemporalJoinTest extends TableTestBase { "Querying a temporal table using 'FOR SYSTEM TIME AS OF' syntax with an expression call " + "'TO_TIMESTAMP(FROM_UNIXTIME(1))' is not supported yet.", classOf[AssertionError]) + + val sqlQuery8 = + s""" + |SELECT * + | FROM OrdersLtz AS o JOIN + | RatesHistoryWithPK FOR SYSTEM_TIME AS OF o.rowtime AS r + | ON o.currency = r.currency + """.stripMargin + expectExceptionThrown( + sqlQuery8, + "Event-Time Temporal Table Join requires same rowtime type in left table and versioned" + + " table, but the rowtime types are TIMESTAMP_LTZ(3) *ROWTIME* and TIMESTAMP(3) *ROWTIME*.", + classOf[ValidationException]) } private def expectExceptionThrown( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala index 77e980530b47f..c1f8d54e336ce 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TemporalTableJoinValidationTest.scala @@ -95,7 +95,7 @@ class TemporalTableJoinValidationTest extends TableTestBase { def testMixedTimeIndicators(): Unit = { expectedException.expect(classOf[ValidationException]) expectedException.expectMessage( - "Non rowtime timeAttribute [TIME ATTRIBUTE(PROCTIME)] passed as the argument " + + "Non rowtime timeAttribute [TIMESTAMP_LTZ(3) *PROCTIME*] passed as the argument " + "to TemporalTableFunction") val rates = ratesHistory.createTemporalTableFunction('rowtime, 'currency) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateHarnessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateHarnessTest.scala index 78ce2db5af7b1..0bb81a78e8790 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateHarnessTest.scala @@ -27,8 +27,10 @@ import org.apache.flink.table.data.{RowData, TimestampData} import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode} import org.apache.flink.table.planner.runtime.utils.TestData -import org.apache.flink.table.runtime.util.{RowDataHarnessAssertor, TimeWindowUtil} +import org.apache.flink.table.runtime.util.RowDataHarnessAssertor import org.apache.flink.table.runtime.util.StreamRecordUtils.binaryRecord +import org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills + import org.apache.flink.types.Row import org.apache.flink.types.RowKind.INSERT @@ -56,7 +58,7 @@ class WindowAggregateHarnessTest(backend: StateBackendMode, shiftTimeZone: ZoneI @Before override def before(): Unit = { super.before() - val dataId = TestValuesTableFactory.registerData(TestData.windowData) + val dataId = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp) tEnv.getConfig.setLocalTimeZone(shiftTimeZone) tEnv.executeSql( s""" @@ -266,9 +268,9 @@ class WindowAggregateHarnessTest(backend: StateBackendMode, shiftTimeZone: ZoneI /** * Ingests testing data, the input schema is [name, double, string, proctime]. - * We follow the test data in [[TestData.windowData]] to have the same produced result. - * The only difference is we don't ingest the late data in this test, so they should produce - * same result. + * We follow the test data in [[TestData.windowDataWithTimestamp]] to have the same produced + * result. The only difference is we don't ingest the late data in this test, so they should + * produce same result. */ private def ingestData( testHarness: KeyedOneInputStreamOperatorTestHarness[RowData, RowData, RowData]): Unit = { @@ -310,7 +312,7 @@ class WindowAggregateHarnessTest(backend: StateBackendMode, shiftTimeZone: ZoneI private def localMills(dateTime: String): TimestampData = { val windowDateTime = LocalDateTime.parse(dateTime).atZone(UTC_ZONE_ID) TimestampData.fromEpochMillis( - TimeWindowUtil.toUtcTimestampMills(windowDateTime.toInstant.toEpochMilli, shiftTimeZone)) + toUtcTimestampMills(windowDateTime.toInstant.toEpochMilli, shiftTimeZone)) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateUseDaylightTimeHarnessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateUseDaylightTimeHarnessTest.scala index 85505da5144fb..ac098d02f196e 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateUseDaylightTimeHarnessTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/harness/WindowAggregateUseDaylightTimeHarnessTest.scala @@ -53,7 +53,7 @@ class WindowAggregateUseDaylightTimeHarnessTest(backend: StateBackendMode, timeZ @Before override def before(): Unit = { super.before() - val dataId = TestValuesTableFactory.registerData(TestData.windowData) + val dataId = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp) tEnv.getConfig.setLocalTimeZone(timeZone.toZoneId) tEnv.executeSql( s""" @@ -117,7 +117,7 @@ class WindowAggregateUseDaylightTimeHarnessTest(backend: StateBackendMode, timeZ expected.add(record("a", 2L, 5.0D, 1L, ts("2021-03-14T03:00:00"), ts("2021-03-14T05:00:00"))) expected.add(record("a", 3L, 5.0D, 1L, ts("2021-03-14T03:00:00"), ts("2021-03-14T06:00:00"))) - // window [2021-11-07T00:00:00, 2021-11-07T02:00:00] windows contains 3 hours data + // [2021-11-07T00:00:00, 2021-11-07T02:00:00] window contains 3 hours data expected.add(record("a", 1L, 3.0D, 1L, ts("2021-11-07T00:00:00"), ts("2021-11-07T01:00:00"))) expected.add(record("a", 3L, 3.0D, 2L, ts("2021-11-07T00:00:00"), ts("2021-11-07T02:00:00"))) expected.add(record("a", 4L, 3.0D, 2L, ts("2021-11-07T00:00:00"), ts("2021-11-07T03:00:00"))) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala index dc8f1d8656881..11b706ac263c4 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala @@ -26,7 +26,7 @@ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.api.internal.TableEnvironmentInternal import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.{ConcatDistinctAggFunction, WeightedAvg} import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_LATE_FIRE_DELAY, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED} -import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode +import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode} import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset import org.apache.flink.table.planner.runtime.utils._ import org.apache.flink.types.Row @@ -36,37 +36,52 @@ import org.junit.Test import org.junit.runner.RunWith import org.junit.runners.Parameterized -import java.math.BigDecimal -import java.time.Duration +import java.time.{Duration, ZoneId} +import java.util import java.util.concurrent.TimeUnit +import org.apache.flink.table.planner.factories.TestValuesTableFactory + +import scala.collection.JavaConversions._ + @RunWith(classOf[Parameterized]) -class GroupWindowITCase(mode: StateBackendMode) +class GroupWindowITCase(mode: StateBackendMode, useTimestampLtz: Boolean) extends StreamingWithStateTestBase(mode) { - val data = List( - (1L, 1, 1d, 1f, new BigDecimal("1"), "Hi", "a"), - (2L, 2, 2d, 2f, new BigDecimal("2"), "Hallo", "a"), - (3L, 2, 2d, 2f, new BigDecimal("2"), "Hello", "a"), - (4L, 5, 5d, 5f, new BigDecimal("5"), "Hello", "a"), - (7L, 3, 3d, 3f, new BigDecimal("3"), "Hello", "b"), - (6L, 5, 5d, 5f, new BigDecimal("5"), "Hello", "a"), - (8L, 3, 3d, 3f, new BigDecimal("3"), "Hello world", "a"), - (16L, 4, 4d, 4f, new BigDecimal("4"), "Hello world", "b"), - (32L, 4, 4d, 4f, new BigDecimal("4"), null.asInstanceOf[String], null.asInstanceOf[String])) + val SHANGHAI_ZONE = ZoneId.of("Asia/Shanghai") + + override def before(): Unit = { + super.before() + + val timestampDataId = TestValuesTableFactory.registerData(TestData.timestampData) + val timestampLtzDataId = TestValuesTableFactory.registerData(TestData.timestampLtzData) + + tEnv.getConfig.setLocalTimeZone(SHANGHAI_ZONE) + tEnv.executeSql( + s""" + |CREATE TABLE testTable ( + | `ts` ${if (useTimestampLtz) "BIGINT" else "STRING"}, + | `int` INT, + | `double` DOUBLE, + | `float` FLOAT, + | `bigdec` DECIMAL(10, 2), + | `string` STRING, + | `name` STRING, + | `rowtime` AS + | ${if (useTimestampLtz) "TO_TIMESTAMP_LTZ(`ts`, 3)" else "TO_TIMESTAMP(`ts`)"}, + | proctime as PROCTIME(), + | WATERMARK for `rowtime` AS `rowtime` - INTERVAL '0.01' SECOND + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '${ if (useTimestampLtz) timestampLtzDataId else timestampDataId}', + | 'failing-source' = 'true' + |) + |""".stripMargin) + } @Test def testEventTimeSlidingWindow(): Unit = { tEnv.registerFunction("concat_distinct_agg", new ConcatDistinctAggFunction()) - - val stream = failingDataSource(data) - .assignTimestampsAndWatermarks( - new TimestampAndWatermarkWithOffset - [(Long, Int, Double, Float, BigDecimal, String, String)](10L)) - val table = stream.toTable(tEnv, - 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'name) - tEnv.registerTable("T1", table) - val sql = """ |SELECT @@ -78,7 +93,7 @@ class GroupWindowITCase(mode: StateBackendMode) | COUNT(`int`), | COUNT(DISTINCT `float`), | concat_distinct_agg(name) - |FROM T1 + |FROM testTable |GROUP BY `string`, HOP(rowtime, INTERVAL '0.004' SECOND, INTERVAL '0.005' SECOND) """.stripMargin @@ -86,36 +101,43 @@ class GroupWindowITCase(mode: StateBackendMode) tEnv.sqlQuery(sql).toAppendStream[Row].addSink(sink) env.execute() - val expected = Seq( - "Hallo,1970-01-01T00:00,1970-01-01T00:00:00.004,1,1,1,1,a", - "Hello world,1970-01-01T00:00:00.004,1970-01-01T00:00:00.008,1,1,1,1,a", - "Hello world,1970-01-01T00:00:00.008,1970-01-01T00:00:00.012,1,1,1,1,a", - "Hello world,1970-01-01T00:00:00.012,1970-01-01T00:00:00.016,1,1,1,1,b", - "Hello world,1970-01-01T00:00:00.016,1970-01-01T00:00:00.020,1,1,1,1,b", - "Hello,1970-01-01T00:00,1970-01-01T00:00:00.004,2,2,2,2,a", - "Hello,1970-01-01T00:00:00.004,1970-01-01T00:00:00.008,3,3,3,2,a|b", - "Hi,1970-01-01T00:00,1970-01-01T00:00:00.004,1,1,1,1,a", - "null,1970-01-01T00:00:00.028,1970-01-01T00:00:00.032,1,1,1,1,null", - "null,1970-01-01T00:00:00.032,1970-01-01T00:00:00.036,1,1,1,1,null") + val expected = if (useTimestampLtz) { + Seq( + "Hallo,1970-01-01T00:00,1969-12-31T16:00:00.004Z,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.004,1969-12-31T16:00:00.008Z,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.008,1969-12-31T16:00:00.012Z,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.012,1969-12-31T16:00:00.016Z,1,1,1,1,b", + "Hello world,1970-01-01T00:00:00.016,1969-12-31T16:00:00.020Z,1,1,1,1,b", + "Hello,1970-01-01T00:00,1969-12-31T16:00:00.004Z,2,2,2,2,a", + "Hello,1970-01-01T00:00:00.004,1969-12-31T16:00:00.008Z,3,3,3,2,a|b", + "Hi,1970-01-01T00:00,1969-12-31T16:00:00.004Z,1,1,1,1,a", + "null,1970-01-01T00:00:00.028,1969-12-31T16:00:00.032Z,1,1,1,1,null", + "null,1970-01-01T00:00:00.032,1969-12-31T16:00:00.036Z,1,1,1,1,null") + } else { + Seq( + "Hallo,1970-01-01T00:00,1970-01-01T00:00:00.004,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.004,1970-01-01T00:00:00.008,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.008,1970-01-01T00:00:00.012,1,1,1,1,a", + "Hello world,1970-01-01T00:00:00.012,1970-01-01T00:00:00.016,1,1,1,1,b", + "Hello world,1970-01-01T00:00:00.016,1970-01-01T00:00:00.020,1,1,1,1,b", + "Hello,1970-01-01T00:00,1970-01-01T00:00:00.004,2,2,2,2,a", + "Hello,1970-01-01T00:00:00.004,1970-01-01T00:00:00.008,3,3,3,2,a|b", + "Hi,1970-01-01T00:00,1970-01-01T00:00:00.004,1,1,1,1,a", + "null,1970-01-01T00:00:00.028,1970-01-01T00:00:00.032,1,1,1,1,null", + "null,1970-01-01T00:00:00.032,1970-01-01T00:00:00.036,1,1,1,1,null") + } assertEquals(expected.sorted, sink.getAppendResults.sorted) } @Test def testCascadingTumbleWindow(): Unit = { - val stream = failingDataSource(data) - .assignTimestampsAndWatermarks( - new TimestampAndWatermarkWithOffset - [(Long, Int, Double, Float, BigDecimal, String, String)](10L)) - val table = stream.toTable(tEnv, - 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'name) - tEnv.registerTable("T1", table) val sql = """ |SELECT SUM(cnt) |FROM ( | SELECT COUNT(1) AS cnt, TUMBLE_ROWTIME(rowtime, INTERVAL '10' SECOND) AS ts - | FROM T1 + | FROM testTable | GROUP BY `int`, `string`, TUMBLE(rowtime, INTERVAL '10' SECOND) |) |GROUP BY TUMBLE(ts, INTERVAL '10' SECOND) @@ -129,8 +151,112 @@ class GroupWindowITCase(mode: StateBackendMode) assertEquals(expected.sorted, sink.getAppendResults.sorted) } + @Test + def testMinMaxWithTumblingWindow(): Unit = { + tEnv.getConfig.getConfiguration.setBoolean("table.exec.emit.early-fire.enabled", true) + tEnv.getConfig.getConfiguration.setString("table.exec.emit.early-fire.delay", "1000 ms") + + val sql = + """ + |SELECT + | MAX(max_ts), + | MIN(min_ts), + | `string` + |FROM( + | SELECT + | `string`, + | `int`, + | MAX(rowtime) as max_ts, + | MIN(rowtime) as min_ts + | FROM testTable + | GROUP BY `string`, `int`, TUMBLE(rowtime, INTERVAL '10' SECOND)) + |GROUP BY `string` + """.stripMargin + val sink = new TestingRetractSink + tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink) + env.execute() + val expected = if (useTimestampLtz) { + Seq( + "1969-12-31T16:00:00.001Z,1969-12-31T16:00:00.001Z,Hi", + "1969-12-31T16:00:00.002Z,1969-12-31T16:00:00.002Z,Hallo", + "1969-12-31T16:00:00.007Z,1969-12-31T16:00:00.003Z,Hello", + "1969-12-31T16:00:00.016Z,1969-12-31T16:00:00.008Z,Hello world", + "1969-12-31T16:00:00.032Z,1969-12-31T16:00:00.032Z,null") + } else { + Seq( + "1970-01-01T00:00:00.001,1970-01-01T00:00:00.001,Hi", + "1970-01-01T00:00:00.002,1970-01-01T00:00:00.002,Hallo", + "1970-01-01T00:00:00.007,1970-01-01T00:00:00.003,Hello", + "1970-01-01T00:00:00.016,1970-01-01T00:00:00.008,Hello world", + "1970-01-01T00:00:00.032,1970-01-01T00:00:00.032,null") + } + assertEquals(expected.sorted, sink.getRetractResults.sorted) + } + + // used to verify compile works normally when constants exists in group window key (FLINK-17553) + @Test + def testWindowAggregateOnConstantValue(): Unit = { + val sql = + """ + |SELECT TUMBLE_END(rowtime, INTERVAL '0.003' SECOND), COUNT(name) + |FROM testTable + | GROUP BY 'a', TUMBLE(rowtime, INTERVAL '0.003' SECOND) + """.stripMargin + val sink = new TestingAppendSink + tEnv.sqlQuery(sql).toAppendStream[Row].addSink(sink) + env.execute() + val expected = Seq( + "1970-01-01T00:00:00.003,2", + "1970-01-01T00:00:00.006,2", + "1970-01-01T00:00:00.009,3", + "1970-01-01T00:00:00.018,1", + "1970-01-01T00:00:00.033,0") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + + @Test + def testProctimeCascadeWindowAgg: Unit = { + val sql = + """ + | SELECT + | cnt AS key, + | TUMBLE_START(pt1, INTERVAL '0.01' SECOND) AS window_start, + | TUMBLE_END(pt1, INTERVAL '0.01' SECOND) AS window_start, + | TUMBLE_PROCTIME(pt1, INTERVAL '0.01' SECOND) as window_proctime, + | MAX(s1) AS v1, + | MAX(e1) AS v2 + | FROM + | (SELECT + | TUMBLE_START(proctime, INTERVAL '0.005' SECOND) as s1, + | TUMBLE_END(proctime, INTERVAL '0.005' SECOND) e1, + | TUMBLE_PROCTIME(proctime, INTERVAL '0.005' SECOND) as pt1, + | COUNT(name) as cnt + | FROM testTable + | GROUP BY 'a', TUMBLE(proctime, INTERVAL '0.005' SECOND) + | ) as T + | GROUP BY cnt, TUMBLE(pt1, INTERVAL '0.01' SECOND) + """.stripMargin + val resolvedSchema = tEnv.sqlQuery(sql).getResolvedSchema + // due to the non-deterministic of proctime() function, the result isn't checked here + assertEquals( + s""" + |( + | `key` BIGINT NOT NULL, + | `window_start` TIMESTAMP(3) NOT NULL, + | `window_start0` TIMESTAMP(3) NOT NULL, + | `window_proctime` TIMESTAMP_LTZ(3) NOT NULL *PROCTIME*, + | `v1` TIMESTAMP(3) NOT NULL, + | `v2` TIMESTAMP(3) NOT NULL + |) + """.stripMargin.trim, + resolvedSchema.toString) + } + @Test def testEventTimeSessionWindow(): Unit = { + if (useTimestampLtz) { + return + } //To verify the "merge" functionality, we create this test with the following characteristics: // 1. set the Parallelism to 1, and have the test data out of order // 2. create a waterMark with 10ms offset to delay the window emission by 10ms @@ -176,6 +302,9 @@ class GroupWindowITCase(mode: StateBackendMode) @Test def testEventTimeTumblingWindowWithAllowLateness(): Unit = { + if (useTimestampLtz) { + return + } // wait 10 millisecond for late elements tEnv.getConfig.setIdleStateRetentionTime( Time.milliseconds(10), Time.minutes(6)) @@ -243,6 +372,9 @@ class GroupWindowITCase(mode: StateBackendMode) @Test def testDistinctAggWithMergeOnEventTimeSessionGroupWindow(): Unit = { + if (useTimestampLtz) { + return + } // create a watermark with 10ms offset to delay the window emission by 10ms to verify merge val sessionWindowTestData = List( (1L, 2, "Hello"), // (1, Hello) - window @@ -279,117 +411,6 @@ class GroupWindowITCase(mode: StateBackendMode) assertEquals(expected.sorted, sink.getAppendResults.sorted) } - @Test - def testMinMaxWithTumblingWindow(): Unit = { - val stream = failingDataSource(data) - .assignTimestampsAndWatermarks( - new TimestampAndWatermarkWithOffset[( - Long, Int, Double, Float, BigDecimal, String, String)](10L)) - val table = - stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'name) - tEnv.registerTable("T1", table) - tEnv.getConfig.getConfiguration.setBoolean("table.exec.emit.early-fire.enabled", true) - tEnv.getConfig.getConfiguration.setString("table.exec.emit.early-fire.delay", "1000 ms") - - val sql = - """ - |SELECT - | MAX(max_ts), - | MIN(min_ts), - | `string` - |FROM( - | SELECT - | `string`, - | `int`, - | MAX(rowtime) as max_ts, - | MIN(rowtime) as min_ts - | FROM T1 - | GROUP BY `string`, `int`, TUMBLE(rowtime, INTERVAL '10' SECOND)) - |GROUP BY `string` - """.stripMargin - val sink = new TestingRetractSink - tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink) - env.execute() - val expected = Seq( - "1970-01-01T00:00:00.001,1970-01-01T00:00:00.001,Hi", - "1970-01-01T00:00:00.002,1970-01-01T00:00:00.002,Hallo", - "1970-01-01T00:00:00.007,1970-01-01T00:00:00.003,Hello", - "1970-01-01T00:00:00.016,1970-01-01T00:00:00.008,Hello world", - "1970-01-01T00:00:00.032,1970-01-01T00:00:00.032,null") - assertEquals(expected.sorted, sink.getRetractResults.sorted) - } - - // used to verify compile works normally when constants exists in group window key (FLINK-17553) - @Test - def testWindowAggregateOnConstantValue(): Unit = { - val stream = failingDataSource(data) - .assignTimestampsAndWatermarks( - new TimestampAndWatermarkWithOffset[( - Long, Int, Double, Float, BigDecimal, String, String)](10L)) - val table = - stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'name) - tEnv.registerTable("T1", table) - val sql = - """ - |SELECT TUMBLE_END(rowtime, INTERVAL '0.003' SECOND), COUNT(name) - |FROM T1 - | GROUP BY 'a', TUMBLE(rowtime, INTERVAL '0.003' SECOND) - """.stripMargin - val sink = new TestingAppendSink - tEnv.sqlQuery(sql).toAppendStream[Row].addSink(sink) - env.execute() - val expected = Seq( - "1970-01-01T00:00:00.003,2", - "1970-01-01T00:00:00.006,2", - "1970-01-01T00:00:00.009,3", - "1970-01-01T00:00:00.018,1", - "1970-01-01T00:00:00.033,0") - assertEquals(expected.sorted, sink.getAppendResults.sorted) - } - - @Test - def testProctimeCascadeWindowAgg: Unit = { - val stream = env.fromCollection(data) - val table = - stream.toTable( - tEnv, 'rowtime, 'int, 'double, 'float, 'bigdec, 'string, 'name, 'proctime.proctime) - tEnv.registerTable("T1", table) - val sql = - """ - | SELECT - | cnt AS key, - | TUMBLE_START(pt1, INTERVAL '0.01' SECOND) AS window_start, - | TUMBLE_END(pt1, INTERVAL '0.01' SECOND) AS window_start, - | TUMBLE_PROCTIME(pt1, INTERVAL '0.01' SECOND) as window_proctime, - | MAX(s1) AS v1, - | MAX(e1) AS v2 - | FROM - | (SELECT - | TUMBLE_START(proctime, INTERVAL '0.005' SECOND) as s1, - | TUMBLE_END(proctime, INTERVAL '0.005' SECOND) e1, - | TUMBLE_PROCTIME(proctime, INTERVAL '0.005' SECOND) as pt1, - | COUNT(name) as cnt - | FROM T1 - | GROUP BY 'a', TUMBLE(proctime, INTERVAL '0.005' SECOND) - | ) as T - | GROUP BY cnt, TUMBLE(pt1, INTERVAL '0.01' SECOND) - """.stripMargin - val resolvedSchema = tEnv.sqlQuery(sql).getResolvedSchema - // due to the non-deterministic of proctime() function, the result isn't checked here - assertEquals( - s""" - |( - | `key` BIGINT NOT NULL, - | `window_start` TIMESTAMP(3) NOT NULL, - | `window_start0` TIMESTAMP(3) NOT NULL, - | `window_proctime` TIMESTAMP_LTZ(3) *PROCTIME*, - | `v1` TIMESTAMP(3) NOT NULL, - | `v2` TIMESTAMP(3) NOT NULL - |) - """.stripMargin.trim, - resolvedSchema.toString) - } - private def withLateFireDelay(tableConfig: TableConfig, interval: Time): Unit = { val intervalInMillis = interval.toMilliseconds val lateFireDelay: Duration = tableConfig.getConfiguration @@ -405,3 +426,17 @@ class GroupWindowITCase(mode: StateBackendMode) TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofMillis(intervalInMillis)) } } + +object GroupWindowITCase { + + @Parameterized.Parameters(name = "StateBackend={0}, UseTimestampLtz = {1}") + def parameters(): util.Collection[Array[java.lang.Object]] = { + Seq[Array[AnyRef]]( + Array(HEAP_BACKEND, java.lang.Boolean.TRUE), + Array(HEAP_BACKEND, java.lang.Boolean.FALSE), + Array(ROCKSDB_BACKEND, java.lang.Boolean.TRUE), + Array(ROCKSDB_BACKEND, java.lang.Boolean.FALSE) + ) + } +} + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala index 63874b32e0a36..7bb92a40cde93 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/MatchRecognizeITCase.scala @@ -25,19 +25,19 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, ScalarFunction} +import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvg import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode import org.apache.flink.table.planner.runtime.utils.TimeTestUtil.EventTimeSourceFunction import org.apache.flink.table.planner.runtime.utils.{StreamingWithStateTestBase, TestingAppendSink, UserDefinedFunctionTestUtils} import org.apache.flink.table.planner.utils.TableTestUtil import org.apache.flink.types.Row - import org.junit.Assert.assertEquals import org.junit.Test import org.junit.runner.RunWith import org.junit.runners.Parameterized - import java.sql.Timestamp +import java.time.{Instant, ZoneId} import java.util.TimeZone import scala.collection.mutable @@ -364,6 +364,71 @@ class MatchRecognizeITCase(backend: StateBackendMode) extends StreamingWithState assertEquals(expected.sorted, sink.getAppendResults.sorted) } + @Test + def testWindowedGroupingAppliedToMatchRecognizeOnLtzRowtime(): Unit = { + val env = StreamExecutionEnvironment.getExecutionEnvironment + val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING) + + val data: Seq[Row] = Seq( + //first window + rowOf("ACME", Instant.ofEpochSecond(1), 1, 1), + rowOf("ACME", Instant.ofEpochSecond(2), 2, 2), + //second window + rowOf("ACME", Instant.ofEpochSecond(3), 1, 4), + rowOf("ACME", Instant.ofEpochSecond(4), 1, 3) + ) + + tEnv.getConfig.setLocalTimeZone(ZoneId.of("Asia/Shanghai")) + + val dataId = TestValuesTableFactory.registerData(data) + tEnv.executeSql( + s""" + |CREATE TABLE Ticker ( + | `symbol` STRING, + | `ts_ltz` TIMESTAMP_LTZ(3), + | `price` INT, + | `tax` INT, + | WATERMARK FOR `ts_ltz` AS `ts_ltz` - INTERVAL '1' SECOND + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '$dataId' + |) + |""".stripMargin) + + val sqlQuery = + s""" + |SELECT + | symbol, + | SUM(price) as price, + | TUMBLE_ROWTIME(matchRowtime, interval '3' second) as rowTime, + | TUMBLE_START(matchRowtime, interval '3' second) as startTime + |FROM Ticker + |MATCH_RECOGNIZE ( + | PARTITION BY symbol + | ORDER BY ts_ltz + | MEASURES + | A.price as price, + | A.tax as tax, + | MATCH_ROWTIME() as matchRowtime + | ONE ROW PER MATCH + | PATTERN (A) + | DEFINE + | A AS A.price > 0 + |) AS T + |GROUP BY symbol, TUMBLE(matchRowtime, interval '3' second) + |""".stripMargin + + val sink = new TestingAppendSink() + val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + result.addSink(sink) + env.execute() + + val expected = List( + "ACME,3,1970-01-01T08:00:02.999,1970-01-01T08:00", + "ACME,2,1970-01-01T08:00:05.999,1970-01-01T08:00:03") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + @Test def testLogicalOffsets(): Unit = { val env = StreamExecutionEnvironment.getExecutionEnvironment diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SourceWatermarkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SourceWatermarkITCase.scala index 7d6e36fc3d0b6..8d3db6d228e37 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SourceWatermarkITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/SourceWatermarkITCase.scala @@ -32,7 +32,7 @@ import org.junit.Assert.assertEquals import org.junit.{Rule, Test} import java.sql.Timestamp -import java.time.LocalDateTime +import java.time.{Instant, LocalDateTime, ZoneId} import scala.collection.JavaConverters._ @@ -92,6 +92,60 @@ class SourceWatermarkITCase extends StreamingTestBase { assertEquals(expectedData.sorted, sink.getAppendResults.sorted) } + @Test + def testSimpleWatermarkOnTimestampLtzPushDown(): Unit = { + val zoneId = ZoneId.of("Asia/Shanghai") + tEnv.getConfig.setLocalTimeZone(zoneId) + val data = Seq( + row(1, 2L, LocalDateTime.parse("2020-11-21T19:00:05.23").atZone(zoneId).toInstant), + row(2, 3L, LocalDateTime.parse("2020-11-21T21:00:05.23").atZone(zoneId).toInstant) + ) + + val dataId = TestValuesTableFactory.registerData(data) + + val ddl = + s""" + | CREATE Table VirtualTable1 ( + | a INT, + | b BIGINT, + | c TIMESTAMP_LTZ(3), + | d as c - INTERVAL '5' second, + | WATERMARK FOR d as d + INTERVAL '5' second + | ) with ( + | 'connector' = 'values', + | 'bounded' = 'false', + | 'enable-watermark-push-down' = 'true', + | 'disable-lookup' = 'true', + | 'data-id' = '$dataId' + | ) + |""".stripMargin + + tEnv.executeSql(ddl) + + val expectedWatermarkOutput = Seq( + "2020-11-21T19:00:05.230", + "2020-11-21T21:00:05.230") + val expectedData = Seq( + "1,2,2020-11-21T11:00:05.230Z", //the utc timestamp of local ts 2020-11-21T19:00:05.230 + "2,3,2020-11-21T13:00:05.230Z" //the utc timestamp of local ts 2020-11-21T21:00:05.230 + ) + + val query = "SELECT a, b, c FROM VirtualTable1" + val result = tEnv.sqlQuery(query).toAppendStream[Row] + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val actualWatermark = TestValuesTableFactory.getWatermarkOutput("VirtualTable1") + .asScala + .map(x => + LocalDateTime.ofInstant(Instant.ofEpochMilli(x.getTimestamp), zoneId).toString) + .toList + + assertEquals(expectedWatermarkOutput, actualWatermark) + assertEquals(expectedData.sorted, sink.getAppendResults.sorted) + } + @Test def testWatermarkWithNestedRow(): Unit = { val data = Seq( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala index 7f3123c5c657d..f2c195bea2850 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala @@ -19,7 +19,6 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.api.scala._ -import org.apache.flink.table.api._ import org.apache.flink.table.api.bridge.scala._ import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5 @@ -30,7 +29,7 @@ import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test import java.sql.Timestamp -import java.time.LocalDateTime +import java.time.{LocalDateTime, ZoneId} import java.util.TimeZone /** @@ -49,6 +48,17 @@ class TimeAttributeITCase extends StreamingTestBase { val dataId: String = TestValuesTableFactory.registerData(data) + val ltzData = List( + rowOf("1970-01-01 08:00:00.001", 1L, 1, 1d), + rowOf("1970-01-01 08:00:00.002", 2L, 1, 2d), + rowOf("1970-01-01 08:00:00.003", 3L, 1, 2d), + rowOf("1970-01-01 08:00:00.004", 4L, 1, 5d), + rowOf("1970-01-01 08:00:00.007", 7L, 1, 3d), + rowOf("1970-01-01 08:00:00.008", 8L, 1, 3d), + rowOf("1970-01-01 08:00:00.016", 16L, 1, 4d)) + + val ltzDataId: String = TestValuesTableFactory.registerData(ltzData) + @Test def testWindowAggregateOnWatermark(): Unit = { val ddl = @@ -83,6 +93,43 @@ class TimeAttributeITCase extends StreamingTestBase { assertEquals(expected.sorted, sink.getAppendResults.sorted) } + @Test + def testWindowAggregateOnTimestampLtzWatermark(): Unit = { + val zoneId = "Asia/Shanghai" + tEnv.getConfig.setLocalTimeZone(ZoneId.of(zoneId)) + val ddl = + s""" + |CREATE TABLE src1 ( + | log_ts STRING, + | ts BIGINT, + | a INT, + | b DOUBLE, + | ltz_ts AS TO_TIMESTAMP_LTZ(ts, 3), + | WATERMARK FOR ltz_ts AS ltz_ts - INTERVAL '0.001' SECOND + |) WITH ( + | 'connector' = 'values', + | 'data-id' = '$ltzDataId' + |) + """.stripMargin + val query = + """ + |SELECT TUMBLE_END(ltz_ts, INTERVAL '0.003' SECOND), COUNT(ts), SUM(b) + |FROM src1 + |GROUP BY TUMBLE(ltz_ts, INTERVAL '0.003' SECOND) + """.stripMargin + tEnv.executeSql(ddl) + val sink = new TestingAppendSink(TimeZone.getTimeZone(zoneId)) + tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink) + env.execute("SQL JOB") + + val expected = Seq( + "1970-01-01T08:00:00.003,2,3.0", + "1970-01-01T08:00:00.006,2,7.0", + "1970-01-01T08:00:00.009,2,6.0", + "1970-01-01T08:00:00.018,1,4.0") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + @Test def testWindowAggregateOnCustomizedWatermark(): Unit = { JavaFunc5.openCalled = false diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala index 0ffbebcc3bb4e..5ece5d966f9f5 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala @@ -37,11 +37,15 @@ import org.junit.runners.Parameterized import org.junit.{Before, Test} import java.util +import java.time.ZoneId import scala.collection.JavaConversions._ @RunWith(classOf[Parameterized]) -class WindowAggregateITCase(aggPhase: AggregatePhaseStrategy, state: StateBackendMode) +class WindowAggregateITCase( + aggPhase: AggregatePhaseStrategy, + state: StateBackendMode, + useTimestampLtz: Boolean) extends StreamingWithStateTestBase(state) { // ------------------------------------------------------------------------------- @@ -127,6 +131,8 @@ class WindowAggregateITCase(aggPhase: AggregatePhaseStrategy, state: StateBacken val CumulateWindowRollupExpectedData = CumulateWindowGroupSetExpectedData + val SHANGHAI_ZONE = ZoneId.of("Asia/Shanghai") + @Before override def before(): Unit = { super.before() @@ -136,27 +142,32 @@ class WindowAggregateITCase(aggPhase: AggregatePhaseStrategy, state: StateBacken env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)) FailingCollectionSource.reset() - val dataId = TestValuesTableFactory.registerData(TestData.windowData) + val timestampDataId = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp) + val timestampLtzDataId = TestValuesTableFactory + .registerData(TestData.windowDataWithLtzInShanghai) + tEnv.executeSql( s""" |CREATE TABLE T1 ( - | `ts` STRING, + | `ts` ${if (useTimestampLtz) "BIGINT" else "STRING"}, | `int` INT, | `double` DOUBLE, | `float` FLOAT, | `bigdec` DECIMAL(10, 2), | `string` STRING, | `name` STRING, - | `rowtime` AS TO_TIMESTAMP(`ts`), + | `rowtime` AS + | ${if (useTimestampLtz) "TO_TIMESTAMP_LTZ(`ts`, 3)" else "TO_TIMESTAMP(`ts`)"}, | WATERMARK for `rowtime` AS `rowtime` - INTERVAL '1' SECOND |) WITH ( | 'connector' = 'values', - | 'data-id' = '$dataId', + | 'data-id' = '${ if (useTimestampLtz) timestampLtzDataId else timestampDataId}', | 'failing-source' = 'true' |) |""".stripMargin) tEnv.createFunction("concat_distinct_agg", classOf[ConcatDistinctAggFunction]) + tEnv.getConfig.setLocalTimeZone(SHANGHAI_ZONE) tEnv.getConfig.getConfiguration.setString( OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhase.toString) @@ -301,13 +312,24 @@ class WindowAggregateITCase(aggPhase: AggregatePhaseStrategy, state: StateBacken tEnv.sqlQuery(sql).toAppendStream[Row].addSink(sink) env.execute() - val expected = Seq( - "a,2020-10-10T00:00,2020-10-10T00:00:05,2020-10-10T00:00:04.999,4", - "a,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-10T00:00:09.999,1", - "b,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-10T00:00:09.999,2", - "b,2020-10-10T00:00:15,2020-10-10T00:00:20,2020-10-10T00:00:19.999,1", - "b,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-10T00:00:34.999,1", - "null,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-10T00:00:34.999,1") + val expected = if (useTimestampLtz) { + Seq( + "a,2020-10-10T00:00,2020-10-10T00:00:05,2020-10-09T16:00:04.999Z,4", + "a,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-09T16:00:09.999Z,1", + "b,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-09T16:00:09.999Z,2", + "b,2020-10-10T00:00:15,2020-10-10T00:00:20,2020-10-09T16:00:19.999Z,1", + "b,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-09T16:00:34.999Z,1", + "null,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-09T16:00:34.999Z,1" + ) + } else { + Seq( + "a,2020-10-10T00:00,2020-10-10T00:00:05,2020-10-10T00:00:04.999,4", + "a,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-10T00:00:09.999,1", + "b,2020-10-10T00:00:05,2020-10-10T00:00:10,2020-10-10T00:00:09.999,2", + "b,2020-10-10T00:00:15,2020-10-10T00:00:20,2020-10-10T00:00:19.999,1", + "b,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-10T00:00:34.999,1", + "null,2020-10-10T00:00:30,2020-10-10T00:00:35,2020-10-10T00:00:34.999,1") + } assertEquals(expected.sorted.mkString("\n"), sink.getAppendResults.sorted.mkString("\n")) } @@ -640,12 +662,14 @@ class WindowAggregateITCase(aggPhase: AggregatePhaseStrategy, state: StateBacken } object WindowAggregateITCase { - @Parameterized.Parameters(name = "AggPhase={0}, StateBackend={1}") + + @Parameterized.Parameters(name = "AggPhase={0}, StateBackend={1}, UseTimestampLtz = {2}") def parameters(): util.Collection[Array[java.lang.Object]] = { Seq[Array[AnyRef]]( - Array(ONE_PHASE, HEAP_BACKEND), - Array(TWO_PHASE, HEAP_BACKEND), - Array(ONE_PHASE, ROCKSDB_BACKEND), - Array(TWO_PHASE, ROCKSDB_BACKEND)) + // 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)) } } diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowDistinctAggregateITCase.scala index 6acab090dafcf..8336946eb515b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowDistinctAggregateITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowDistinctAggregateITCase.scala @@ -152,7 +152,7 @@ class WindowDistinctAggregateITCase( env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)) FailingCollectionSource.reset() - val dataId = TestValuesTableFactory.registerData(TestData.windowData) + val dataId = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp) tEnv.executeSql( s""" |CREATE TABLE T1 ( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowRankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowRankITCase.scala index f3f26f8037b93..9bada417fe920 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowRankITCase.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowRankITCase.scala @@ -46,7 +46,7 @@ class WindowRankITCase(mode: StateBackendMode) env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)) FailingCollectionSource.reset() - val dataId = TestValuesTableFactory.registerData(TestData.windowData) + val dataId = TestValuesTableFactory.registerData(TestData.windowDataWithTimestamp) tEnv.executeSql( s""" |CREATE TABLE T1 ( diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala index 952ec61e2e355..c756677b9e54b 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TestData.scala @@ -587,7 +587,7 @@ object TestData { changelogRow("-D", "Yen", JLong.valueOf(1L)) ) - val windowData: Seq[Row] = List( + val windowDataWithTimestamp: Seq[Row] = List( row("2020-10-10 00:00:01", 1, 1d, 1f, new JBigDecimal("1.11"), "Hi", "a"), row("2020-10-10 00:00:02", 2, 2d, 2f, new JBigDecimal("2.22"), "Comment#1", "a"), row("2020-10-10 00:00:03", 2, 2d, 2f, new JBigDecimal("2.22"), "Comment#1", "a"), @@ -603,6 +603,63 @@ object TestData { row("2020-10-10 00:00:32", 7, 7d, 7f, new JBigDecimal("7.77"), null, null), row("2020-10-10 00:00:34", 1, 3d, 3f, new JBigDecimal("3.33"), "Comment#3", "b")) + val shanghaiZone = ZoneId.of("Asia/Shanghai") + val windowDataWithLtzInShanghai: Seq[Row] = List( + row(toEpochMills("2020-10-10T00:00:01", shanghaiZone), + 1, 1d, 1f, new JBigDecimal("1.11"), "Hi", "a"), + row(toEpochMills("2020-10-10T00:00:02", shanghaiZone), + 2, 2d, 2f, new JBigDecimal("2.22"), "Comment#1", "a"), + row(toEpochMills("2020-10-10T00:00:03", shanghaiZone), + 2, 2d, 2f, new JBigDecimal("2.22"), "Comment#1", "a"), + row(toEpochMills("2020-10-10T00:00:04", shanghaiZone), + 5, 5d, 5f, new JBigDecimal("5.55"), null, "a"), + row(toEpochMills("2020-10-10T00:00:07", shanghaiZone), + 3, 3d, 3f, null, "Hello", "b"), + row(toEpochMills("2020-10-10T00:00:06", shanghaiZone), + 6, 6d, 6f, new JBigDecimal("6.66"), "Hi", "b"), // out of order + row(toEpochMills("2020-10-10T00:00:08", shanghaiZone), + 3, null, 3f, new JBigDecimal("3.33"), "Comment#2", "a"), + row(toEpochMills("2020-10-10T00:00:04", shanghaiZone), + 5, 5d, null, new JBigDecimal("5.55"), "Hi", "a"), // late event + row(toEpochMills("2020-10-10T00:00:16", shanghaiZone), + 4, 4d, 4f, new JBigDecimal("4.44"), "Hi", "b"), + row(toEpochMills("2020-10-10T00:00:32", shanghaiZone), + 7, 7d, 7f, new JBigDecimal("7.77"), null, null), + row(toEpochMills("2020-10-10T00:00:34", shanghaiZone), + 1, 3d, 3f, new JBigDecimal("3.33"), "Comment#3", "b")) + + val timestampData: Seq[Row] = List( + row("1970-01-01 00:00:00.001", 1, 1d, 1f, new JBigDecimal("1"), "Hi", "a"), + row("1970-01-01 00:00:00.002", 2, 2d, 2f, new JBigDecimal("2"), "Hallo", "a"), + row("1970-01-01 00:00:00.003", 2, 2d, 2f, new JBigDecimal("2"), "Hello", "a"), + row("1970-01-01 00:00:00.004", 5, 5d, 5f, new JBigDecimal("5"), "Hello", "a"), + row("1970-01-01 00:00:00.007", 3, 3d, 3f, new JBigDecimal("3"), "Hello", "b"), + row("1970-01-01 00:00:00.006", 5, 5d, 5f, new JBigDecimal("5"), "Hello", "a"), + row("1970-01-01 00:00:00.008", 3, 3d, 3f, new JBigDecimal("3"), "Hello world", "a"), + row("1970-01-01 00:00:00.016", 4, 4d, 4f, new JBigDecimal("4"), "Hello world", "b"), + row("1970-01-01 00:00:00.032", 4, 4d, 4f, + new JBigDecimal("4"), null.asInstanceOf[String], null.asInstanceOf[String])) + + val timestampLtzData: Seq[Row] = List( + row(toEpochMills("1970-01-01T00:00:00.001", shanghaiZone), + 1, 1d, 1f, new JBigDecimal("1"), "Hi", "a"), + row(toEpochMills("1970-01-01T00:00:00.002", shanghaiZone), + 2, 2d, 2f, new JBigDecimal("2"), "Hallo", "a"), + row(toEpochMills("1970-01-01T00:00:00.003", shanghaiZone), + 2, 2d, 2f, new JBigDecimal("2"), "Hello", "a"), + row(toEpochMills("1970-01-01T00:00:00.004", shanghaiZone), + 5, 5d, 5f, new JBigDecimal("5"), "Hello", "a"), + row(toEpochMills("1970-01-01T00:00:00.007", shanghaiZone), + 3, 3d, 3f, new JBigDecimal("3"), "Hello", "b"), + row(toEpochMills("1970-01-01T00:00:00.006", shanghaiZone), + 5, 5d, 5f, new JBigDecimal("5"), "Hello", "a"), + row(toEpochMills("1970-01-01T00:00:00.008", shanghaiZone), + 3, 3d, 3f, new JBigDecimal("3"), "Hello world", "a"), + row(toEpochMills("1970-01-01T00:00:00.016", shanghaiZone), + 4, 4d, 4f, new JBigDecimal("4"), "Hello world", "b"), + row(toEpochMills("1970-01-01T00:00:00.032", shanghaiZone), + 4, 4d, 4f, new JBigDecimal("4"), null.asInstanceOf[String], null.asInstanceOf[String])) + val fullDataTypesData: Seq[Row] = { val bools = List(true, false, true, false, null) val bytes = List(Byte.MaxValue, Byte.MinValue, 0.byteValue(), 5.byteValue(), null) diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala index 518323d7bceb1..78f704381685f 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/DateTimeTestUtil.scala @@ -23,8 +23,7 @@ import org.apache.flink.table.runtime.functions.SqlDateTimeUtils import org.apache.calcite.avatica.util.DateTimeUtils import org.apache.calcite.avatica.util.DateTimeUtils.dateStringToUnixDate - -import java.time.{LocalDate, LocalDateTime, LocalTime} +import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId} object DateTimeTestUtil { @@ -51,4 +50,8 @@ object DateTimeTestUtil { SqlDateTimeUtils.toTimestampData(s).toLocalDateTime } } + + def toEpochMills(s: String, zone: ZoneId): Long = { + LocalDateTime.parse(s).atZone(zone).toInstant.toEpochMilli + } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/LocalSlicingWindowAggOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/LocalSlicingWindowAggOperator.java index 6f0a7d7f3f837..0190706422890 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/LocalSlicingWindowAggOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/LocalSlicingWindowAggOperator.java @@ -31,13 +31,19 @@ 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.LocalAggRecordsCombiner; -import org.apache.flink.table.runtime.operators.window.TimeWindow; import org.apache.flink.table.runtime.operators.window.combines.WindowCombineFunction; import org.apache.flink.table.runtime.operators.window.slicing.ClockService; import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner; import org.apache.flink.table.runtime.typeutils.AbstractRowDataSerializer; import org.apache.flink.table.runtime.typeutils.PagedTypeSerializer; +import java.time.ZoneId; +import java.util.TimeZone; + +import static org.apache.flink.table.runtime.operators.window.TimeWindow.getWindowStartWithOffset; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; + /** * The operator used for local window aggregation. * @@ -54,6 +60,16 @@ public class LocalSlicingWindowAggOperator extends AbstractStreamOperator keySer, AbstractRowDataSerializer inputSer, - GeneratedNamespaceAggsHandleFunction genAggsHandler) { + GeneratedNamespaceAggsHandleFunction genAggsHandler, + ZoneId shiftTimezone) { this( keySelector, sliceAssigner, new RecordsWindowBuffer.Factory(keySer, inputSer), - new LocalAggRecordsCombiner.Factory(genAggsHandler, keySer)); + new LocalAggRecordsCombiner.Factory(genAggsHandler, keySer), + shiftTimezone); } public LocalSlicingWindowAggOperator( RowDataKeySelector keySelector, SliceAssigner sliceAssigner, WindowBuffer.Factory windowBufferFactory, - WindowCombineFunction.LocalFactory combinerFactory) { + WindowCombineFunction.LocalFactory combinerFactory, + ZoneId shiftTimezone) { this.keySelector = keySelector; this.sliceAssigner = sliceAssigner; this.windowInterval = sliceAssigner.getSliceEndInterval(); this.windowBufferFactory = windowBufferFactory; this.combinerFactory = combinerFactory; + this.shiftTimezone = shiftTimezone; + this.useDayLightSaving = TimeZone.getTimeZone(shiftTimezone).useDaylightTime(); } @Override @@ -111,7 +132,8 @@ public void open() throws Exception { getContainingTask(), getContainingTask().getEnvironment().getMemoryManager(), computeMemorySize(), - localCombiner); + localCombiner, + shiftTimezone); } @Override @@ -128,9 +150,11 @@ public void processWatermark(Watermark mark) throws Exception { if (mark.getTimestamp() > currentWatermark) { currentWatermark = mark.getTimestamp(); if (currentWatermark >= nextTriggerWatermark) { - // we only need to call advanceProgress() when currentWatermark may trigger window + // we only need to call advanceProgress() when current watermark may trigger window windowBuffer.advanceProgress(currentWatermark); - nextTriggerWatermark = getNextTriggerWatermark(currentWatermark, windowInterval); + nextTriggerWatermark = + getNextTriggerWatermark( + currentWatermark, windowInterval, shiftTimezone, useDayLightSaving); } } super.processWatermark(mark); @@ -176,9 +200,24 @@ private long computeMemorySize() { // Utilities // ------------------------------------------------------------------------ /** Method to get the next watermark to trigger window. */ - private static long getNextTriggerWatermark(long currentWatermark, long interval) { - long start = TimeWindow.getWindowStartWithOffset(currentWatermark, 0L, interval); - long triggerWatermark = start + interval - 1; + private static long getNextTriggerWatermark( + long currentWatermark, long interval, ZoneId shiftTimezone, boolean useDayLightSaving) { + if (currentWatermark == Long.MAX_VALUE) { + return currentWatermark; + } + + long triggerWatermark; + // consider the DST timezone + if (useDayLightSaving) { + long utcWindowStart = + getWindowStartWithOffset( + toUtcTimestampMills(currentWatermark, shiftTimezone), 0L, interval); + triggerWatermark = toEpochMillsForTimer(utcWindowStart + interval - 1, shiftTimezone); + } else { + long start = getWindowStartWithOffset(currentWatermark, 0L, interval); + triggerWatermark = start + interval - 1; + } + if (triggerWatermark > currentWatermark) { return triggerWatermark; } else { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorBuilder.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorBuilder.java index 302faebb2d8c3..44bdd2c9b3fc9 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorBuilder.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorBuilder.java @@ -171,6 +171,6 @@ public SlicingWindowAggOperatorBuilder countStarIndex(int indexOfCountStart) { throw new IllegalArgumentException( "assigner must be instance of SliceUnsharedAssigner or SliceSharedAssigner."); } - return new SlicingWindowOperator<>(windowProcessor, shiftTimeZone); + return new SlicingWindowOperator<>(windowProcessor); } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/RecordsWindowBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/RecordsWindowBuffer.java index 3cdeb9c5726b8..ed68f0a580755 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/RecordsWindowBuffer.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/RecordsWindowBuffer.java @@ -30,8 +30,11 @@ import org.apache.flink.table.runtime.util.collections.binary.WindowBytesMultiMap; import java.io.EOFException; +import java.time.ZoneId; import java.util.Iterator; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + /** * An implementation of {@link WindowBuffer} that buffers input elements in a {@link * WindowBytesMultiMap} and combines buffered elements into state when flushing. @@ -42,8 +45,9 @@ public final class RecordsWindowBuffer implements WindowBuffer { private final WindowBytesMultiMap recordsBuffer; private final WindowKey reuseWindowKey; private final AbstractRowDataSerializer recordSerializer; + private final ZoneId shiftTimeZone; - private long minTriggerTime = Long.MAX_VALUE; + private long minSliceEnd = Long.MAX_VALUE; public RecordsWindowBuffer( Object operatorOwner, @@ -51,13 +55,15 @@ public RecordsWindowBuffer( long memorySize, WindowCombineFunction combineFunction, PagedTypeSerializer keySer, - AbstractRowDataSerializer inputSer) { + AbstractRowDataSerializer inputSer, + ZoneId shiftTimeZone) { this.combineFunction = combineFunction; this.recordsBuffer = new WindowBytesMultiMap( operatorOwner, memoryManager, memorySize, keySer, inputSer.getArity()); this.recordSerializer = inputSer; this.reuseWindowKey = new WindowKeySerializer(keySer).createInstance(); + this.shiftTimeZone = shiftTimeZone; } @Override @@ -65,7 +71,7 @@ public void addElement(RowData key, long sliceEnd, RowData element) throws Excep // 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. - minTriggerTime = Math.min(sliceEnd - 1, minTriggerTime); + minSliceEnd = Math.min(sliceEnd, minSliceEnd); reuseWindowKey.replace(sliceEnd, key); LookupInfo> lookup = recordsBuffer.lookup(reuseWindowKey); @@ -81,7 +87,7 @@ public void addElement(RowData key, long sliceEnd, RowData element) throws Excep @Override public void advanceProgress(long progress) throws Exception { - if (progress >= minTriggerTime) { + if (isWindowFired(minSliceEnd, progress, shiftTimeZone)) { // there should be some window to be fired, flush buffer to state first flush(); } @@ -97,7 +103,7 @@ public void flush() throws Exception { } recordsBuffer.reset(); // reset trigger time - minTriggerTime = Long.MAX_VALUE; + minSliceEnd = Long.MAX_VALUE; } } @@ -129,9 +135,16 @@ public WindowBuffer create( Object operatorOwner, MemoryManager memoryManager, long memorySize, - WindowCombineFunction combineFunction) { + WindowCombineFunction combineFunction, + ZoneId shiftTimeZone) { return new RecordsWindowBuffer( - operatorOwner, memoryManager, memorySize, combineFunction, keySer, inputSer); + operatorOwner, + memoryManager, + memorySize, + combineFunction, + keySer, + inputSer, + shiftTimeZone); } } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/WindowBuffer.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/WindowBuffer.java index c1175e80581fd..7b954f83c20de 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/WindowBuffer.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/buffers/WindowBuffer.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.io.Serializable; +import java.time.ZoneId; /** * A buffer that buffers data in memory and flushes many values to state together at a time to avoid @@ -83,13 +84,15 @@ interface Factory extends Serializable { * @param memoryManager the manager that governs memory by Flink framework * @param memorySize the managed memory size can be used by this operator * @param combineFunction the combine function used to combine buffered data into state + * @param shiftTimeZone the shit timezone of the window * @throws IOException thrown if the buffer can't be opened */ WindowBuffer create( Object operatorOwner, MemoryManager memoryManager, long memorySize, - WindowCombineFunction combineFunction) + WindowCombineFunction combineFunction, + ZoneId shiftTimeZone) throws IOException; } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/AggRecordsCombiner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/AggRecordsCombiner.java index 38ce951d05499..2ab8c6309be6f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/AggRecordsCombiner.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/AggRecordsCombiner.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; 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.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.combines.WindowCombineFunction; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService; import org.apache.flink.table.runtime.operators.window.state.StateKeyContext; import org.apache.flink.table.runtime.operators.window.state.WindowState; import org.apache.flink.table.runtime.operators.window.state.WindowValueState; @@ -45,7 +45,7 @@ public final class AggRecordsCombiner implements WindowCombineFunction { /** The service to register event-time or processing-time timers. */ - private final InternalTimerService timerService; + private final WindowTimerService timerService; /** Context to switch current key for states. */ private final StateKeyContext keyContext; @@ -69,7 +69,7 @@ public final class AggRecordsCombiner implements WindowCombineFunction { private final boolean isEventTime; public AggRecordsCombiner( - InternalTimerService timerService, + WindowTimerService timerService, StateKeyContext keyContext, WindowValueState accState, NamespaceAggsHandleFunction aggregator, @@ -129,7 +129,7 @@ record = recordSerializer.copy(record); // step 5: register timer for current window if (isEventTime) { - timerService.registerEventTimeTimer(window, window - 1); + timerService.registerEventTimeWindowTimer(window); } // we don't need register processing-time timer, because we already register them // per-record in AbstractWindowAggProcessor.processElement() @@ -165,7 +165,7 @@ public Factory( @Override public WindowCombineFunction create( RuntimeContext runtimeContext, - InternalTimerService timerService, + WindowTimerService timerService, KeyedStateBackend stateBackend, WindowState windowState, boolean isEventTime) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/GlobalAggAccCombiner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/GlobalAggAccCombiner.java index 180d74fbfaf4a..f12c9a631cdd6 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/GlobalAggAccCombiner.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/combines/GlobalAggAccCombiner.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; 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.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.combines.WindowCombineFunction; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService; import org.apache.flink.table.runtime.operators.window.state.StateKeyContext; import org.apache.flink.table.runtime.operators.window.state.WindowState; import org.apache.flink.table.runtime.operators.window.state.WindowValueState; @@ -46,7 +46,7 @@ public final class GlobalAggAccCombiner implements WindowCombineFunction { /** The service to register event-time or processing-time timers. */ - private final InternalTimerService timerService; + private final WindowTimerService timerService; /** Context to switch current key for states. */ private final StateKeyContext keyContext; @@ -67,7 +67,7 @@ public final class GlobalAggAccCombiner implements WindowCombineFunction { private final TypeSerializer keySerializer; public GlobalAggAccCombiner( - InternalTimerService timerService, + WindowTimerService timerService, StateKeyContext keyContext, WindowValueState accState, NamespaceAggsHandleFunction localAggregator, @@ -116,7 +116,7 @@ public void combine(WindowKey windowKey, Iterator localAccs) throws Exc accState.update(window, stateAcc); // step 3: register timer for current window - timerService.registerEventTimeTimer(window, window - 1); + timerService.registerEventTimeWindowTimer(window); } @Override @@ -153,7 +153,7 @@ public Factory( @Override public WindowCombineFunction create( RuntimeContext runtimeContext, - InternalTimerService timerService, + WindowTimerService timerService, KeyedStateBackend stateBackend, WindowState windowState, boolean isEventTime) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java index edd352790d3ed..6a76c69ac5eb0 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/AbstractWindowAggProcessor.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.runtime.state.internal.InternalValueState; -import org.apache.flink.streaming.api.operators.InternalTimerService; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; import org.apache.flink.table.runtime.dataview.PerWindowStateDataViewStore; @@ -34,11 +33,14 @@ import org.apache.flink.table.runtime.operators.window.slicing.ClockService; import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner; import org.apache.flink.table.runtime.operators.window.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerServiceImpl; import org.apache.flink.table.runtime.operators.window.state.WindowValueState; -import org.apache.flink.table.runtime.util.TimeWindowUtil; import java.time.ZoneId; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + /** A base implementation of {@link SlicingWindowProcessor} for window aggregate. */ public abstract class AbstractWindowAggProcessor implements SlicingWindowProcessor { private static final long serialVersionUID = 1L; @@ -49,13 +51,7 @@ public abstract class AbstractWindowAggProcessor implements SlicingWindowProcess protected final SliceAssigner sliceAssigner; protected final TypeSerializer accSerializer; protected final boolean isEventTime; - - /** - * The shift timezone of the window, if the proctime or rowtime type is TIMESTAMP_LTZ, the shift - * timezone is the timezone user configured in TableConfig, other cases the timezone is UTC - * which means never shift when assigning windows. - */ - protected final ZoneId shiftTimeZone; + private final ZoneId shiftTimeZone; // ---------------------------------------------------------------------------------------- @@ -65,7 +61,7 @@ public abstract class AbstractWindowAggProcessor implements SlicingWindowProcess protected transient ClockService clockService; - protected transient InternalTimerService timerService; + protected transient WindowTimerService windowTimerService; protected transient NamespaceAggsHandleFunction aggregator; @@ -104,7 +100,7 @@ public void open(Context context) throws Exception { this.windowState = new WindowValueState<>((InternalValueState) state); this.clockService = ClockService.of(ctx.getTimerService()); - this.timerService = ctx.getTimerService(); + this.windowTimerService = new WindowTimerServiceImpl(ctx.getTimerService(), shiftTimeZone); this.aggregator = genAggsHandler.newInstance(ctx.getRuntimeContext().getUserCodeClassLoader()); this.aggregator.open( @@ -113,7 +109,7 @@ public void open(Context context) throws Exception { final WindowCombineFunction combineFunction = combineFactory.create( ctx.getRuntimeContext(), - ctx.getTimerService(), + windowTimerService, ctx.getKeyedStateBackend(), windowState, isEventTime); @@ -122,7 +118,8 @@ public void open(Context context) throws Exception { ctx.getOperatorOwner(), ctx.getMemoryManager(), ctx.getMemorySize(), - combineFunction); + combineFunction, + shiftTimeZone); this.reuseOutput = new JoinedRowData(); this.currentProgress = Long.MIN_VALUE; @@ -133,10 +130,9 @@ public boolean 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 - timerService.registerProcessingTimeTimer( - sliceEnd, TimeWindowUtil.toEpochMillsForTimer(sliceEnd - 1, shiftTimeZone)); + windowTimerService.registerProcessingTimeWindowTimer(sliceEnd); } - if (isEventTime && sliceEnd - 1 <= currentProgress) { + if (isEventTime && isWindowFired(sliceEnd, currentProgress, shiftTimeZone)) { // element is late and should be dropped return true; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java index 9bb634c06e15c..6f59fbc5a6cd9 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/aggregate/window/processors/SliceSharedWindowAggProcessor.java @@ -26,7 +26,6 @@ import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner; import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigners; import org.apache.flink.table.runtime.operators.window.slicing.SliceSharedAssigner; -import org.apache.flink.table.runtime.util.TimeWindowUtil; import javax.annotation.Nullable; @@ -85,11 +84,9 @@ public void fireWindow(Long windowEnd) throws Exception { if (nextWindowEndOptional.isPresent()) { long nextWindowEnd = nextWindowEndOptional.get(); if (sliceSharedAssigner.isEventTime()) { - timerService.registerEventTimeTimer(nextWindowEnd, nextWindowEnd - 1); + windowTimerService.registerEventTimeWindowTimer(nextWindowEnd); } else { - timerService.registerProcessingTimeTimer( - nextWindowEnd, - TimeWindowUtil.toEpochMillsForTimer(nextWindowEnd - 1, shiftTimeZone)); + windowTimerService.registerProcessingTimeWindowTimer(nextWindowEnd); } } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java index 1374852d5f77d..da89489eac50e 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/WindowRankOperatorBuilder.java @@ -155,7 +155,8 @@ public WindowRankOperatorBuilder windowEndIndex(int windowEndIndex) { rankStart, rankEnd, outputRankNumber, - windowEndIndex); - return new SlicingWindowOperator<>(windowProcessor, shiftTimeZone); + windowEndIndex, + shiftTimeZone); + return new SlicingWindowOperator<>(windowProcessor); } } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/combines/TopNRecordsCombiner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/combines/TopNRecordsCombiner.java index b8d6772c55fb4..f33757e554bf0 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/combines/TopNRecordsCombiner.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/combines/TopNRecordsCombiner.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; 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.generated.GeneratedRecordComparator; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.rank.TopNBuffer; import org.apache.flink.table.runtime.operators.window.combines.WindowCombineFunction; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService; import org.apache.flink.table.runtime.operators.window.state.StateKeyContext; import org.apache.flink.table.runtime.operators.window.state.WindowMapState; import org.apache.flink.table.runtime.operators.window.state.WindowState; @@ -50,7 +50,7 @@ public final class TopNRecordsCombiner implements WindowCombineFunction { /** The service to register event-time or processing-time timers. */ - private final InternalTimerService timerService; + private final WindowTimerService timerService; /** Context to switch current key for states. */ private final StateKeyContext keyContext; @@ -80,7 +80,7 @@ public final class TopNRecordsCombiner implements WindowCombineFunction { private final boolean isEventTime; public TopNRecordsCombiner( - InternalTimerService timerService, + WindowTimerService timerService, StateKeyContext keyContext, WindowMapState> dataState, Comparator sortKeyComparator, @@ -144,7 +144,7 @@ public void combine(WindowKey windowKey, Iterator records) throws Excep } // step 3: register timer for current window if (isEventTime) { - timerService.registerEventTimeTimer(window, window - 1); + timerService.registerEventTimeWindowTimer(window); } // we don't need register processing-time timer, because we already register them // per-record in AbstractWindowAggProcessor.processElement() @@ -185,7 +185,7 @@ public Factory( @Override public WindowCombineFunction create( RuntimeContext runtimeContext, - InternalTimerService timerService, + WindowTimerService timerService, KeyedStateBackend stateBackend, WindowState windowState, boolean isEventTime) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java index 18bd821b6ffb2..c45b401e9b901 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/rank/window/processors/WindowRankProcessor.java @@ -32,9 +32,12 @@ import org.apache.flink.table.runtime.operators.rank.TopNBuffer; import org.apache.flink.table.runtime.operators.window.combines.WindowCombineFunction; import org.apache.flink.table.runtime.operators.window.slicing.SlicingWindowProcessor; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerService; +import org.apache.flink.table.runtime.operators.window.slicing.WindowTimerServiceImpl; import org.apache.flink.table.runtime.operators.window.state.WindowMapState; import org.apache.flink.types.RowKind; +import java.time.ZoneId; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; @@ -42,6 +45,8 @@ import java.util.List; import java.util.Map; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.isWindowFired; + /** An window rank processor. */ public final class WindowRankProcessor implements SlicingWindowProcessor { private static final long serialVersionUID = 1L; @@ -60,6 +65,7 @@ public final class WindowRankProcessor implements SlicingWindowProcessor { private final long rankEnd; private final boolean outputRankNumber; private final int windowEndIndex; + private final ZoneId shiftTimeZone; // ---------------------------------------------------------------------------------------- @@ -67,6 +73,8 @@ public final class WindowRankProcessor implements SlicingWindowProcessor { private transient Context ctx; + private transient WindowTimerService windowTimerService; + private transient WindowBuffer windowBuffer; /** state schema: [key, window_end, sort key, records]. */ @@ -84,7 +92,8 @@ public WindowRankProcessor( long rankStart, long rankEnd, boolean outputRankNumber, - int windowEndIndex) { + int windowEndIndex, + ZoneId shiftTimeZone) { this.inputSerializer = inputSerializer; this.generatedSortKeyComparator = genSortKeyComparator; this.sortKeySerializer = sortKeySerializer; @@ -94,6 +103,7 @@ public WindowRankProcessor( this.rankEnd = rankEnd; this.outputRankNumber = outputRankNumber; this.windowEndIndex = windowEndIndex; + this.shiftTimeZone = shiftTimeZone; } @Override @@ -112,13 +122,15 @@ public void open(Context context) throws Exception { MapState> state = ctx.getKeyedStateBackend() .getOrCreateKeyedState(namespaceSerializer, mapStateDescriptor); + + this.windowTimerService = new WindowTimerServiceImpl(ctx.getTimerService(), shiftTimeZone); this.windowState = new WindowMapState<>( (InternalMapState>) state); final WindowCombineFunction combineFunction = combineFactory.create( ctx.getRuntimeContext(), - ctx.getTimerService(), + windowTimerService, ctx.getKeyedStateBackend(), windowState, true); @@ -127,7 +139,8 @@ public void open(Context context) throws Exception { ctx.getOperatorOwner(), ctx.getMemoryManager(), ctx.getMemorySize(), - combineFunction); + combineFunction, + shiftTimeZone); this.reuseOutput = new JoinedRowData(); this.reuseRankRow = new GenericRowData(1); @@ -137,7 +150,7 @@ public void open(Context context) throws Exception { @Override public boolean processElement(RowData key, RowData element) throws Exception { long sliceEnd = element.getLong(windowEndIndex); - if (sliceEnd - 1 <= currentProgress) { + if (isWindowFired(sliceEnd, currentProgress, shiftTimeZone)) { // element is late and should be dropped return true; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java index 34cbe5a1564c8..77344b791de26 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperator.java @@ -62,6 +62,7 @@ import java.util.Collection; import static java.util.Objects.requireNonNull; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -419,9 +420,7 @@ public void onProcessingTime(InternalTimer timer) throws Exception { } if (!windowAssigner.isEventTime()) { - windowFunction.cleanWindowIfNeeded( - triggerContext.window, - TimeWindowUtil.toUtcTimestampMills(timer.getTimestamp(), shiftTimeZone)); + windowFunction.cleanWindowIfNeeded(triggerContext.window, timer.getTimestamp()); } } @@ -434,7 +433,7 @@ public void onProcessingTime(InternalTimer timer) throws Exception { * @param window the window whose state to discard */ private void registerCleanupTimer(W window) { - long cleanupTime = cleanupTime(window); + long cleanupTime = toEpochMillsForTimer(cleanupTime(window), shiftTimeZone); if (cleanupTime == Long.MAX_VALUE) { // don't set a GC timer for "end of time" return; @@ -495,6 +494,11 @@ public long currentWatermark() { return internalTimerService.currentWatermark(); } + @Override + public ZoneId getShiftTimeZone() { + return shiftTimeZone; + } + @Override public RowData getWindowAccumulators(W window) throws Exception { windowState.setCurrentNamespace(window); @@ -530,7 +534,7 @@ public void clearTrigger(W window) throws Exception { @Override public void deleteCleanupTimer(W window) throws Exception { - long cleanupTime = cleanupTime(window); + long cleanupTime = toEpochMillsForTimer(cleanupTime(window), shiftTimeZone); if (cleanupTime == Long.MAX_VALUE) { // no need to clean up because we didn't set one return; @@ -569,8 +573,7 @@ boolean onElement(RowData row, long timestamp) throws Exception { } boolean onProcessingTime(long time) throws Exception { - return trigger.onProcessingTime( - TimeWindowUtil.toUtcTimestampMills(time, shiftTimeZone), window); + return trigger.onProcessingTime(time, window); } boolean onEventTime(long time) throws Exception { @@ -598,8 +601,7 @@ public MetricGroup getMetricGroup() { @Override public void registerProcessingTimeTimer(long time) { - internalTimerService.registerProcessingTimeTimer( - window, TimeWindowUtil.toEpochMillsForTimer(time, shiftTimeZone)); + internalTimerService.registerProcessingTimeTimer(window, time); } @Override @@ -609,8 +611,7 @@ public void registerEventTimeTimer(long time) { @Override public void deleteProcessingTimeTimer(long time) { - internalTimerService.deleteProcessingTimeTimer( - window, TimeWindowUtil.toEpochMillsForTimer(time, shiftTimeZone)); + internalTimerService.deleteProcessingTimeTimer(window, time); } @Override @@ -618,6 +619,11 @@ public void deleteEventTimeTimer(long time) { internalTimerService.deleteEventTimeTimer(window, time); } + @Override + public ZoneId getShiftTimeZone() { + return shiftTimeZone; + } + public void clear() throws Exception { trigger.clear(window); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java index e721857e34694..943142ae05180 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/WindowOperatorBuilder.java @@ -73,7 +73,7 @@ public class WindowOperatorBuilder { protected long allowedLateness = 0L; protected boolean produceUpdates = false; protected int rowtimeIndex = -1; - protected ZoneId shiftTimeZone; + protected ZoneId shiftTimeZone = ZoneId.of("UTC"); public static WindowOperatorBuilder builder() { return new WindowOperatorBuilder(); diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/combines/WindowCombineFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/combines/WindowCombineFunction.java index 2d3a3cfa9063e..c12a2476f9e0b 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/combines/WindowCombineFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/combines/WindowCombineFunction.java @@ -21,8 +21,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.RuntimeContext; 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.slicing.WindowTimerService; import org.apache.flink.table.runtime.operators.window.state.WindowState; import org.apache.flink.table.runtime.util.WindowKey; import org.apache.flink.util.Collector; @@ -67,7 +67,7 @@ interface Factory extends Serializable { */ WindowCombineFunction create( RuntimeContext runtimeContext, - InternalTimerService timerService, + WindowTimerService timerService, KeyedStateBackend stateBackend, WindowState windowState, boolean isEventTime) diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java index c646b62d3f413..b1dab8b0bdcf7 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/InternalWindowProcessFunction.java @@ -27,8 +27,11 @@ import org.apache.flink.table.runtime.operators.window.triggers.Trigger; import java.io.Serializable; +import java.time.ZoneId; import java.util.Collection; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; + /** * The internal interface for functions that process over grouped windows. * @@ -106,7 +109,7 @@ public void close() throws Exception {} /** Returns {@code true} if the given time is the cleanup time for the given window. */ protected final boolean isCleanupTime(W window, long time) { - return time == cleanupTime(window); + return time == toEpochMillsForTimer(cleanupTime(window), ctx.getShiftTimeZone()); } /** @@ -114,7 +117,9 @@ protected final boolean isCleanupTime(W window, long time) { * the given window. */ protected boolean isWindowLate(W window) { - return (windowAssigner.isEventTime() && (cleanupTime(window) <= ctx.currentWatermark())); + return (windowAssigner.isEventTime() + && (toEpochMillsForTimer(cleanupTime(window), ctx.getShiftTimeZone()) + <= ctx.currentWatermark())); } /** @@ -158,6 +163,9 @@ S getPartitionedState(StateDescriptor stateDescriptor) /** Returns the current event-time watermark. */ long currentWatermark(); + /** Returns the shifted timezone of the window. */ + ZoneId getShiftTimeZone(); + /** Gets the accumulators of the given window. */ RowData getWindowAccumulators(W window) throws Exception; diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java index 76535cca11f4b..ad085c26e5e85 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/internal/MergingWindowProcessFunction.java @@ -30,6 +30,8 @@ import java.util.Collection; import java.util.List; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; + /** * The implementation of {@link InternalWindowProcessFunction} for {@link MergingWindowAssigner}. * @@ -132,8 +134,10 @@ public void merge( Collection stateWindowsToBeMerged) throws Exception { + long mergeResultMaxTs = + toEpochMillsForTimer(mergeResult.maxTimestamp(), ctx.getShiftTimeZone()); if ((windowAssigner.isEventTime() - && mergeResult.maxTimestamp() + allowedLateness <= ctx.currentWatermark())) { + && mergeResultMaxTs + allowedLateness <= ctx.currentWatermark())) { throw new UnsupportedOperationException( "The end timestamp of an " + "event-time window cannot become earlier than the current watermark " @@ -142,7 +146,7 @@ public void merge( + " window: " + mergeResult); } else if (!windowAssigner.isEventTime() - && mergeResult.maxTimestamp() <= ctx.currentProcessingTime()) { + && mergeResultMaxTs <= ctx.currentProcessingTime()) { throw new UnsupportedOperationException( "The end timestamp of a " + "processing-time window cannot become earlier than the current processing time " diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SliceAssigners.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SliceAssigners.java index 0b8ce56145c4b..40b99fb617d6f 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SliceAssigners.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SliceAssigners.java @@ -21,7 +21,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.operators.window.TimeWindow; -import org.apache.flink.table.runtime.util.TimeWindowUtil; import org.apache.flink.util.IterableIterator; import org.apache.flink.util.MathUtils; @@ -36,6 +35,7 @@ import java.util.Optional; import java.util.function.Supplier; +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills; import static org.apache.flink.util.Preconditions.checkArgument; /** Utilities to create {@link SliceAssigner}s. */ @@ -515,14 +515,10 @@ protected AbstractSliceAssigner(int rowtimeIndex, ZoneId shiftTimeZone) { public final long assignSliceEnd(RowData element, ClockService clock) { final long timestamp; if (rowtimeIndex >= 0) { - timestamp = - TimeWindowUtil.toUtcTimestampMills( - element.getLong(rowtimeIndex), shiftTimeZone); + timestamp = toUtcTimestampMills(element.getLong(rowtimeIndex), shiftTimeZone); } else { // in processing time mode - timestamp = - TimeWindowUtil.toUtcTimestampMills( - clock.currentProcessingTime(), shiftTimeZone); + timestamp = toUtcTimestampMills(clock.currentProcessingTime(), shiftTimeZone); } return assignSliceEnd(timestamp); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SlicingWindowOperator.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SlicingWindowOperator.java index f57cb8a4fa5f5..516f09b199c55 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SlicingWindowOperator.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/SlicingWindowOperator.java @@ -40,9 +40,6 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.operators.TableStreamOperator; import org.apache.flink.table.runtime.operators.aggregate.window.processors.SliceSharedWindowAggProcessor; -import org.apache.flink.table.runtime.util.TimeWindowUtil; - -import java.time.ZoneId; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -108,12 +105,6 @@ public final class SlicingWindowOperator extends TableStreamOperator windowProcessor; - /** - * The shift timezone of the window, if the proctime or rowtime type is TIMESTAMP_LTZ, the shift - * timezone is the timezone user configured in TableConfig, other cases the timezone is UTC - * which means never shift when assigning windows. - */ - private final ZoneId shiftTimeZone; // ------------------------------------------------------------------------ /** This is used for emitting elements with a given timestamp. */ @@ -136,9 +127,8 @@ public final class SlicingWindowOperator extends TableStreamOperator watermarkLatency; - public SlicingWindowOperator(SlicingWindowProcessor windowProcessor, ZoneId shiftTimeZone) { + public SlicingWindowOperator(SlicingWindowProcessor windowProcessor) { this.windowProcessor = windowProcessor; - this.shiftTimeZone = shiftTimeZone; setChainingStrategy(ChainingStrategy.ALWAYS); } @@ -226,13 +216,11 @@ public void onEventTime(InternalTimer timer) throws Exception { @Override public void onProcessingTime(InternalTimer timer) throws Exception { - long timestamp = TimeWindowUtil.toUtcTimestampMills(timer.getTimestamp(), shiftTimeZone); - - if (timestamp > lastTriggeredProcessingTime) { + if (timer.getTimestamp() > lastTriggeredProcessingTime) { // similar to the watermark advance, // we need to notify WindowProcessor first to flush buffer into state - lastTriggeredProcessingTime = timestamp; - windowProcessor.advanceProgress(timestamp); + lastTriggeredProcessingTime = timer.getTimestamp(); + windowProcessor.advanceProgress(timer.getTimestamp()); // timers registered in advanceProgress() should always be smaller than current timer // so, it should be safe to trigger current timer straightforwards. } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerService.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerService.java new file mode 100644 index 0000000000000..6280b121b9ef3 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerService.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.slicing; + +import org.apache.flink.annotation.Internal; + +import java.time.ZoneId; + +/** + * Interface for working with window time and timers which considers timezone for window splitting. + * + * @param Type of the window namespace to which timers are scoped. + */ +@Internal +public interface WindowTimerService { + + /** + * The shift timezone of the window, if the proctime or rowtime type is TIMESTAMP_LTZ, the shift + * timezone is the timezone user configured in TableConfig, other cases the timezone is UTC + * which means never shift when assigning windows. + */ + ZoneId getShiftTimeZone(); + + /** Returns the current processing time. */ + long currentProcessingTime(); + + /** Returns the current event-time watermark. */ + long currentWatermark(); + + /** + * Registers a window timer to be fired when processing time passes the window. The window you + * pass here will be provided when the timer fires. + */ + void registerProcessingTimeWindowTimer(W window); + + /** + * Registers a window timer to be fired when event time watermark passes the window. The window + * you pass here will be provided when the timer fires. + */ + void registerEventTimeWindowTimer(W window); +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerServiceImpl.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerServiceImpl.java new file mode 100644 index 0000000000000..836ab3df63670 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/slicing/WindowTimerServiceImpl.java @@ -0,0 +1,64 @@ +/* + * 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.slicing; + +import org.apache.flink.streaming.api.operators.InternalTimerService; +import org.apache.flink.table.runtime.util.TimeWindowUtil; + +import java.time.ZoneId; + +/** Simple Implements of {@link WindowTimerService}. */ +public class WindowTimerServiceImpl implements WindowTimerService { + + private final InternalTimerService internalTimerService; + private final ZoneId shiftTimeZone; + + public WindowTimerServiceImpl( + InternalTimerService internalTimerService, ZoneId shiftTimeZone) { + this.internalTimerService = internalTimerService; + this.shiftTimeZone = shiftTimeZone; + } + + @Override + public ZoneId getShiftTimeZone() { + return shiftTimeZone; + } + + @Override + public long currentProcessingTime() { + return internalTimerService.currentProcessingTime(); + } + + @Override + public long currentWatermark() { + return internalTimerService.currentWatermark(); + } + + @Override + public void registerProcessingTimeWindowTimer(Long window) { + internalTimerService.registerProcessingTimeTimer( + window, TimeWindowUtil.toEpochMillsForTimer(window - 1, shiftTimeZone)); + } + + @Override + public void registerEventTimeWindowTimer(Long window) { + internalTimerService.registerEventTimeTimer( + window, TimeWindowUtil.toEpochMillsForTimer(window - 1, shiftTimeZone)); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java index fab5ef1727b64..b92b8f38ef194 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/EventTimeTriggers.java @@ -58,7 +58,7 @@ public static AfterEndOfWindow afterEndOfWindow() { * * @see org.apache.flink.streaming.api.watermark.Watermark */ - public static final class AfterEndOfWindow extends Trigger { + public static final class AfterEndOfWindow extends WindowTrigger { private static final long serialVersionUID = -6379468077823588591L; @@ -85,8 +85,6 @@ public Trigger withLateFirings(Trigger lateFirings) { } } - private TriggerContext ctx; - @Override public void open(TriggerContext ctx) throws Exception { this.ctx = ctx; @@ -94,11 +92,11 @@ public void open(TriggerContext ctx) throws Exception { @Override public boolean onElement(Object element, long timestamp, W window) throws Exception { - if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { + if (triggerTime(window) <= ctx.getCurrentWatermark()) { // if the watermark is already past the window fire immediately return true; } else { - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); return false; } } @@ -110,12 +108,12 @@ public boolean onProcessingTime(long time, W window) throws Exception { @Override public boolean onEventTime(long time, W window) throws Exception { - return time == window.maxTimestamp(); + return time == triggerTime(window); } @Override public void clear(W window) throws Exception { - ctx.deleteEventTimeTimer(window.maxTimestamp()); + ctx.deleteEventTimeTimer(triggerTime(window)); } @Override @@ -125,7 +123,7 @@ public boolean canMerge() { @Override public void onMerge(W window, OnMergeContext mergeContext) throws Exception { - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); } @Override @@ -138,7 +136,8 @@ public String toString() { * A composite {@link Trigger} that consist of AfterEndOfWindow and a early trigger and late * trigger. */ - public static final class AfterEndOfWindowEarlyAndLate extends Trigger { + public static final class AfterEndOfWindowEarlyAndLate + extends WindowTrigger { private static final long serialVersionUID = -800582945577030338L; @@ -146,8 +145,6 @@ public static final class AfterEndOfWindowEarlyAndLate extends private final Trigger lateTrigger; private final ValueStateDescriptor hasFiredOnTimeStateDesc; - private transient TriggerContext ctx; - AfterEndOfWindowEarlyAndLate(Trigger earlyTrigger, Trigger lateTrigger) { this.earlyTrigger = earlyTrigger; this.lateTrigger = lateTrigger; @@ -174,7 +171,7 @@ public boolean onElement(Object element, long timestamp, W window) throws Except // is Long.MIN_VALUE but the window is already in the late phase. return lateTrigger != null && lateTrigger.onElement(element, timestamp, window); } else { - if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { + if (triggerTime(window) <= ctx.getCurrentWatermark()) { // we are in the late phase // if there is no late trigger then we fire on every late element @@ -183,7 +180,7 @@ public boolean onElement(Object element, long timestamp, W window) throws Except return true; } else { // we are in the early phase - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); return earlyTrigger != null && earlyTrigger.onElement(element, timestamp, window); } @@ -210,7 +207,7 @@ public boolean onEventTime(long time, W window) throws Exception { // late fire return lateTrigger != null && lateTrigger.onEventTime(time, window); } else { - if (time == window.maxTimestamp()) { + if (time == triggerTime(window)) { // fire on time and update state hasFiredState.update(true); return true; @@ -239,7 +236,7 @@ public void onMerge(W window, OnMergeContext mergeContext) throws Exception { // we assume that the new merged window has not fired yet its on-time timer. ctx.getPartitionedState(hasFiredOnTimeStateDesc).update(false); - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); } @Override @@ -250,7 +247,7 @@ public void clear(W window) throws Exception { if (lateTrigger != null) { lateTrigger.clear(window); } - ctx.deleteEventTimeTimer(window.maxTimestamp()); + ctx.deleteEventTimeTimer(triggerTime(window)); ctx.getPartitionedState(hasFiredOnTimeStateDesc).clear(); } @@ -268,7 +265,7 @@ public String toString() { } /** A composite {@link Trigger} that consist of AfterEndOfWindow and a late trigger. */ - public static final class AfterEndOfWindowNoLate extends Trigger { + public static final class AfterEndOfWindowNoLate extends WindowTrigger { private static final long serialVersionUID = -4334481808648361926L; @@ -288,7 +285,6 @@ public Trigger withLateFirings(Trigger lateFirings) { // early trigger is always not null private final Trigger earlyTrigger; - private TriggerContext ctx; private AfterEndOfWindowNoLate(Trigger earlyTrigger) { checkNotNull(earlyTrigger); @@ -303,12 +299,12 @@ public void open(TriggerContext ctx) throws Exception { @Override public boolean onElement(Object element, long timestamp, W window) throws Exception { - if (window.maxTimestamp() <= ctx.getCurrentWatermark()) { + if (triggerTime(window) <= ctx.getCurrentWatermark()) { // the on-time firing return true; } else { // this is an early element so register the timer and let the early trigger decide - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); return earlyTrigger.onElement(element, timestamp, window); } } @@ -320,7 +316,7 @@ public boolean onProcessingTime(long time, W window) throws Exception { @Override public boolean onEventTime(long time, W window) throws Exception { - return time == window.maxTimestamp() || earlyTrigger.onEventTime(time, window); + return time == triggerTime(window) || earlyTrigger.onEventTime(time, window); } @Override @@ -330,13 +326,13 @@ public boolean canMerge() { @Override public void onMerge(W window, OnMergeContext mergeContext) throws Exception { - ctx.registerEventTimeTimer(window.maxTimestamp()); + ctx.registerEventTimeTimer(triggerTime(window)); earlyTrigger.onMerge(window, mergeContext); } @Override public void clear(W window) throws Exception { - ctx.deleteEventTimeTimer(window.maxTimestamp()); + ctx.deleteEventTimeTimer(triggerTime(window)); earlyTrigger.clear(window); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java index ef4d3d1f44c90..af36b42af51b2 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/ProcessingTimeTriggers.java @@ -73,15 +73,13 @@ public static AfterFirstElementPeriodic every(Duration tim * * @param type of window */ - public static final class AfterFirstElementPeriodic extends Trigger { + public static final class AfterFirstElementPeriodic extends WindowTrigger { private static final long serialVersionUID = -4710472821577125673L; private final long interval; private final ReducingStateDescriptor nextFiringStateDesc; - private transient TriggerContext ctx; - AfterFirstElementPeriodic(long interval) { checkArgument(interval > 0); this.interval = interval; @@ -171,7 +169,7 @@ public Long reduce(Long value1, Long value2) throws Exception { * A {@link Trigger} that fires once the current system time passes the end of the window to * which a pane belongs. */ - public static final class AfterEndOfWindow extends Trigger { + public static final class AfterEndOfWindow extends WindowTrigger { private static final long serialVersionUID = 2369815941792574642L; /** @@ -183,8 +181,6 @@ public AfterEndOfWindowNoLate withEarlyFirings(Trigger earlyFirings) { return new AfterEndOfWindowNoLate<>(earlyFirings); } - private TriggerContext ctx; - @Override public void open(TriggerContext ctx) throws Exception { this.ctx = ctx; @@ -192,13 +188,13 @@ public void open(TriggerContext ctx) throws Exception { @Override public boolean onElement(Object element, long timestamp, W window) throws Exception { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); + ctx.registerProcessingTimeTimer(triggerTime(window)); return false; } @Override public boolean onProcessingTime(long time, W window) throws Exception { - return time == window.maxTimestamp(); + return time == triggerTime(window); } @Override @@ -208,7 +204,7 @@ public boolean onEventTime(long time, W window) throws Exception { @Override public void clear(W window) throws Exception { - ctx.deleteProcessingTimeTimer(window.maxTimestamp()); + ctx.deleteProcessingTimeTimer(triggerTime(window)); } @Override @@ -218,7 +214,7 @@ public boolean canMerge() { @Override public void onMerge(W window, OnMergeContext mergeContext) throws Exception { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); + ctx.registerProcessingTimeTimer(triggerTime(window)); } @Override @@ -228,13 +224,12 @@ public String toString() { } /** A composite {@link Trigger} that consist of AfterEndOfWindow and a early trigger. */ - public static final class AfterEndOfWindowNoLate extends Trigger { + public static final class AfterEndOfWindowNoLate extends WindowTrigger { private static final long serialVersionUID = 2310050856564792734L; // early trigger is always not null private final Trigger earlyTrigger; - private TriggerContext ctx; AfterEndOfWindowNoLate(Trigger earlyTrigger) { checkNotNull(earlyTrigger); @@ -249,13 +244,13 @@ public void open(TriggerContext ctx) throws Exception { @Override public boolean onElement(Object element, long timestamp, W window) throws Exception { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); + ctx.registerProcessingTimeTimer(triggerTime(window)); return earlyTrigger.onElement(element, timestamp, window); } @Override public boolean onProcessingTime(long time, W window) throws Exception { - return time == window.maxTimestamp() || earlyTrigger.onProcessingTime(time, window); + return time == triggerTime(window) || earlyTrigger.onProcessingTime(time, window); } @Override @@ -270,13 +265,13 @@ public boolean canMerge() { @Override public void onMerge(W window, OnMergeContext mergeContext) throws Exception { - ctx.registerProcessingTimeTimer(window.maxTimestamp()); + ctx.registerProcessingTimeTimer(triggerTime(window)); earlyTrigger.onMerge(window, mergeContext); } @Override public void clear(W window) throws Exception { - ctx.deleteProcessingTimeTimer(window.maxTimestamp()); + ctx.deleteProcessingTimeTimer(triggerTime(window)); earlyTrigger.clear(window); } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java index d26faf394dfee..57956271768e4 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/Trigger.java @@ -27,6 +27,7 @@ import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner; import java.io.Serializable; +import java.time.ZoneId; /** * A {@code Trigger} determines when a pane of a window should be evaluated to emit the results for @@ -167,6 +168,9 @@ public interface TriggerContext { /** Delete the event-time trigger for the given time. */ void deleteEventTimeTimer(long time); + /** Returns the shifted timezone. */ + ZoneId getShiftTimeZone(); + /** * Retrieves a {@link State} object that can be used to interact with fault-tolerant state * that is scoped to the window and key of the current trigger invocation. diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/WindowTrigger.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/WindowTrigger.java new file mode 100644 index 0000000000000..2eb7492678ff1 --- /dev/null +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/window/triggers/WindowTrigger.java @@ -0,0 +1,43 @@ +/* + * 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.triggers; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.runtime.operators.window.Window; + +import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer; + +/** A {@code WindowTrigger} determines when a window should be evaluated to emit the results. */ +@Internal +public abstract class WindowTrigger extends Trigger { + + /** + * The {@link org.apache.flink.table.runtime.operators.window.triggers.Trigger.TriggerContext} + * of the window trigger. + */ + protected transient TriggerContext ctx; + + /** + * Returns the trigger time of the window, this should be called after TriggerContext + * initialized. + */ + protected long triggerTime(W window) { + return toEpochMillsForTimer(window.maxTimestamp(), ctx.getShiftTimeZone()); + } +} diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java index da7d46db32074..7be6a30a8ac33 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/types/PlannerTypeUtils.java @@ -98,6 +98,7 @@ public static boolean isInteroperable(LogicalType t1, LogicalType t2) { && t2.getTypeRoot().getFamilies().contains(BINARY_STRING)) { return true; } + if (t1.getTypeRoot() != t2.getTypeRoot()) { return false; } diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java index f319c7695c8e1..1f90d48947bbe 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/typeutils/TypeCheckUtils.java @@ -36,6 +36,7 @@ import static org.apache.flink.table.types.logical.LogicalTypeRoot.ROW; import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE; import static org.apache.flink.table.types.logical.LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE; +import static org.apache.flink.table.types.logical.utils.LogicalTypeChecks.isRowtimeAttribute; /** Utils for type. */ public class TypeCheckUtils { @@ -53,8 +54,8 @@ public static boolean isTimePoint(LogicalType type) { } public static boolean isRowTime(LogicalType type) { - return type instanceof TimestampType - && ((TimestampType) type).getKind() == TimestampKind.ROWTIME; + return (type instanceof TimestampType || type instanceof LocalZonedTimestampType) + && isRowtimeAttribute(type); } public static boolean isProcTime(LogicalType type) { diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/TimeWindowUtil.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/TimeWindowUtil.java index 4c850012b45a8..17f1a44ad6031 100644 --- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/TimeWindowUtil.java +++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/TimeWindowUtil.java @@ -42,14 +42,15 @@ public class TimeWindowUtil { * Convert a epoch mills to timestamp mills which can describe a locate date time. * *

For example: The timestamp string of epoch mills 5 in UTC+8 is 1970-01-01 08:00:05, the - * timestamp mills is 8 * 60 * 60 * 100 + 5. + * timestamp mills is 8 * 60 * 60 * 1000 + 5. * * @param epochMills the epoch mills. * @param shiftTimeZone the timezone that the given timestamp mills has been shifted. * @return the mills which can describe the local timestamp string in given timezone. */ public static long toUtcTimestampMills(long epochMills, ZoneId shiftTimeZone) { - if (UTC_ZONE_ID.equals(shiftTimeZone)) { + // Long.MAX_VALUE is a flag of max watermark, directly return it + if (UTC_ZONE_ID.equals(shiftTimeZone) || Long.MAX_VALUE == epochMills) { return epochMills; } LocalDateTime localDateTime = @@ -65,27 +66,49 @@ public static long toUtcTimestampMills(long epochMills, ZoneId shiftTimeZone) { * @return the epoch mills. */ public static long toEpochMillsForTimer(long utcTimestampMills, ZoneId shiftTimeZone) { - if (shiftTimeZone.equals(UTC_ZONE_ID)) { + // Long.MAX_VALUE is a flag of max watermark, directly return it + if (UTC_ZONE_ID.equals(shiftTimeZone) || Long.MAX_VALUE == utcTimestampMills) { return utcTimestampMills; } if (TimeZone.getTimeZone(shiftTimeZone).useDaylightTime()) { /* - * return the larger epoch mills if the time is leaving the DST. + * return the first skipped epoch mills as timer time if the time is coming the DST. + * eg. Los_Angele has no timestamp 2021-03-14 02:00:00 when coming DST. + *

+             *  2021-03-14 00:00:00 -> epoch1 = 1615708800000L;
+             *  2021-03-14 01:00:00 -> epoch2 = 1615712400000L;
+             *  2021-03-14 03:00:00 -> epoch3 = 1615716000000L;  skip one hour (2021-03-14 02:00:00)
+             *  2021-03-14 04:00:00 -> epoch4 = 1615719600000L;
+             *
+             * we should use the epoch3 to register timer for window that end with
+             *  [2021-03-14 02:00:00, 2021-03-14 03:00:00] to ensure the window can be fired
+             *  immediately once the window passed.
+             *
+             * 
+             *  2021-03-14 00:00:00 -> epoch0 = 1615708800000L;
+             *  2021-03-14 01:00:00 -> epoch1 = 1615712400000L;
+             *  2021-03-14 02:00:00 -> epoch3 = 1615716000000L; register 1615716000000L(epoch3)
+             *  2021-03-14 02:59:59 -> epoch3 = 1615719599000L; register 1615716000000L(epoch3)
+             *  2021-03-14 03:00:00 -> epoch3 = 1615716000000L;
+             */
+
+            /*
+             * return the larger epoch mills as timer time if the time is leaving the DST.
              *  eg. Los_Angeles has two timestamp 2021-11-07 01:00:00 when leaving DST.
              * 
-             *  2021-11-07 00:00:00 -> epoch0  = 1636268400000L;  2021-11-07 00:00:00
-             *  2021-11-07 01:00:00 -> epoch1  = 1636272000000L;  the first local timestamp 2021-11-07 01:00:00
-             *  2021-11-07 01:00:00 -> epoch2  = 1636275600000L;  back to local timestamp  2021-11-07 01:00:00
-             *  2021-11-07 02:00:00 -> epoch3  = 1636279200000L;  2021-11-07 02:00:00
+             *  2021-11-07 00:00:00 -> epoch0 = 1636268400000L;  2021-11-07 00:00:00
+             *  2021-11-07 01:00:00 -> epoch1 = 1636272000000L;  the first local timestamp 2021-11-07 01:00:00
+             *  2021-11-07 01:00:00 -> epoch2 = 1636275600000L;  back to local timestamp  2021-11-07 01:00:00
+             *  2021-11-07 02:00:00 -> epoch3 = 1636279200000L;  2021-11-07 02:00:00
              *
              * we should use the epoch1 + 1 hour to register timer to ensure the two hours' data can
              * be fired properly.
              *
              * 
-             *  2021-11-07 00:00:00 => long epoch0 = 1636268400000L;
-             *  2021-11-07 01:00:00 => long epoch1 = 1636272000000L; register 1636275600000L(epoch2)
-             *  2021-11-07 02:00:00 => long epoch3 = 1636279200000L;
+             *  2021-11-07 00:00:00 -> epoch0 = 1636268400000L;
+             *  2021-11-07 01:00:00 -> epoch1 = 1636272000000L; register 1636275600000L(epoch2)
+             *  2021-11-07 02:00:00 -> epoch3 = 1636279200000L;
              */
             LocalDateTime utcTimestamp =
                     LocalDateTime.ofInstant(Instant.ofEpochMilli(utcTimestampMills), UTC_ZONE_ID);
@@ -96,8 +119,13 @@ public static long toEpochMillsForTimer(long utcTimestampMills, ZoneId shiftTime
                             .atZone(shiftTimeZone)
                             .toInstant()
                             .toEpochMilli();
+
+            boolean hasNoEpoch = t1 == t2;
             boolean hasTwoEpochs = t2 - t1 > MILLS_PER_HOUR;
-            if (hasTwoEpochs) {
+
+            if (hasNoEpoch) {
+                return t1 - t1 % MILLS_PER_HOUR;
+            } else if (hasTwoEpochs) {
                 return t1 + MILLS_PER_HOUR;
             } else {
                 return t1;
@@ -117,7 +145,8 @@ public static long toEpochMillsForTimer(long utcTimestampMills, ZoneId shiftTime
      * @return the epoch mills.
      */
     public static long toEpochMills(long utcTimestampMills, ZoneId shiftTimeZone) {
-        if (UTC_ZONE_ID.equals(shiftTimeZone)) {
+        // Long.MAX_VALUE is a flag of max watermark, directly return it
+        if (UTC_ZONE_ID.equals(shiftTimeZone) || Long.MAX_VALUE == utcTimestampMills) {
             return utcTimestampMills;
         }
         LocalDateTime utcTimestamp =
@@ -133,4 +162,22 @@ public static ZoneId getShiftTimeZone(LogicalType timeAttributeType, TableConfig
         boolean needShiftTimeZone = timeAttributeType instanceof LocalZonedTimestampType;
         return needShiftTimeZone ? tableConfig.getLocalTimeZone() : UTC_ZONE_ID;
     }
+
+    /**
+     * Returns the window should fired or not on current progress.
+     *
+     * @param windowEnd the end of the time window.
+     * @param currentProgress current progress of the window operator, it is processing time under
+     *     proctime, it is watermark value under rowtime.
+     * @param shiftTimeZone the shifted timezone of the time window.
+     */
+    public static boolean isWindowFired(
+            long windowEnd, long currentProgress, ZoneId shiftTimeZone) {
+        // Long.MAX_VALUE is a flag of min window end, directly return false
+        if (windowEnd == Long.MAX_VALUE) {
+            return false;
+        }
+        long windowTriggerTime = toEpochMillsForTimer(windowEnd - 1, shiftTimeZone);
+        return currentProgress >= windowTriggerTime;
+    }
 }
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java
index f948cda3a1a9c..cf2a28c56f169 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/aggregate/window/SlicingWindowAggOperatorTest.java
@@ -45,24 +45,37 @@
 import org.apache.flink.table.utils.HandwrittenSelectorUtil;
 
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 import java.time.Duration;
 import java.time.LocalDateTime;
 import java.time.ZoneId;
 import java.time.ZoneOffset;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.apache.flink.core.testutils.FlinkMatchers.containsMessage;
 import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord;
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
 
 /** Tests for window aggregate operators created by {@link SlicingWindowAggOperatorBuilder}. */
+@RunWith(Parameterized.class)
 public class SlicingWindowAggOperatorTest {
 
+    private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC");
+    private static final ZoneId SHANGHAI_ZONE_ID = ZoneId.of("Asia/Shanghai");
+    private ZoneId shiftTimeZone;
+
+    public SlicingWindowAggOperatorTest(ZoneId shiftTimeZone) {
+        this.shiftTimeZone = shiftTimeZone;
+    }
+
     private static final RowType INPUT_ROW_TYPE =
             new RowType(
                     Arrays.asList(
@@ -99,18 +112,16 @@ public class SlicingWindowAggOperatorTest {
                     OUTPUT_TYPES,
                     new GenericRowRecordSortComparator(0, new VarCharType(VarCharType.MAX_LENGTH)));
 
-    private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC");
-
     @Test
     public void testEventTimeHoppingWindows() throws Exception {
         final SliceAssigner assigner =
                 SliceAssigners.hopping(
-                        2, UTC_ZONE_ID, Duration.ofSeconds(3), Duration.ofSeconds(1));
+                        2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1));
         final SumAndCountAggsFunction aggsFunction = new SumAndCountAggsFunction(assigner);
         SlicingWindowOperator operator =
                 SlicingWindowAggOperatorBuilder.builder()
                         .inputSerializer(INPUT_ROW_SER)
-                        .shiftTimeZone(UTC_ZONE_ID)
+                        .shiftTimeZone(shiftTimeZone)
                         .keySerializer(KEY_SER)
                         .assigner(assigner)
                         .aggregate(wrapGenerated(aggsFunction), ACC_SER)
@@ -139,21 +150,21 @@ public void testEventTimeHoppingWindows() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 1000L));
 
         testHarness.processWatermark(new Watermark(999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, -2000L, 1000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(-2000L), localMills(1000L)));
         expectedOutput.add(new Watermark(999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(1999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, -1000L, 2000L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, -1000L, 2000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(-1000L), localMills(2000L)));
+        expectedOutput.add(insertRecord("key2", 3L, 3L, localMills(-1000L), localMills(2000L)));
         expectedOutput.add(new Watermark(1999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(2999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 3000L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 3000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), localMills(3000L)));
+        expectedOutput.add(insertRecord("key2", 3L, 3L, localMills(0L), localMills(3000L)));
         expectedOutput.add(new Watermark(2999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -170,7 +181,7 @@ public void testEventTimeHoppingWindows() throws Exception {
         testHarness.open();
 
         testHarness.processWatermark(new Watermark(3999));
-        expectedOutput.add(insertRecord("key2", 5L, 5L, 1000L, 4000L));
+        expectedOutput.add(insertRecord("key2", 5L, 5L, localMills(1000L), localMills(4000L)));
         expectedOutput.add(new Watermark(3999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -179,7 +190,7 @@ public void testEventTimeHoppingWindows() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 3500L));
 
         testHarness.processWatermark(new Watermark(4999));
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 2000L, 5000L));
+        expectedOutput.add(insertRecord("key2", 2L, 2L, localMills(2000L), localMills(5000L)));
         expectedOutput.add(new Watermark(4999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -188,7 +199,7 @@ public void testEventTimeHoppingWindows() throws Exception {
         testHarness.processElement(insertRecord("key1", 1, 4999L));
 
         testHarness.processWatermark(new Watermark(5999));
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 3000L, 6000L));
+        expectedOutput.add(insertRecord("key2", 2L, 2L, localMills(3000L), localMills(6000L)));
         expectedOutput.add(new Watermark(5999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -211,16 +222,7 @@ public void testEventTimeHoppingWindows() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeHoppingWindowsInUTC() throws Exception {
-        testProcessingTimeHoppingWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeHoppingWindowsInShanghai() throws Exception {
-        testProcessingTimeHoppingWindows(ZoneId.of("Asia/Shanghai"));
-    }
-
-    private void testProcessingTimeHoppingWindows(ZoneId shiftTimeZone) throws Exception {
+    public void testProcessingTimeHoppingWindows() throws Exception {
         final SliceAssigner assigner =
                 SliceAssigners.hopping(-1, shiftTimeZone, Duration.ofHours(3), Duration.ofHours(1));
         final SumAndCountAggsFunction aggsFunction = new SumAndCountAggsFunction(assigner);
@@ -344,12 +346,12 @@ private void testProcessingTimeHoppingWindows(ZoneId shiftTimeZone) throws Excep
     public void testEventTimeCumulativeWindows() throws Exception {
         final SliceAssigner assigner =
                 SliceAssigners.cumulative(
-                        2, UTC_ZONE_ID, Duration.ofSeconds(3), Duration.ofSeconds(1));
+                        2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1));
         final SumAndCountAggsFunction aggsFunction = new SumAndCountAggsFunction(assigner);
         SlicingWindowOperator operator =
                 SlicingWindowAggOperatorBuilder.builder()
                         .inputSerializer(INPUT_ROW_SER)
-                        .shiftTimeZone(UTC_ZONE_ID)
+                        .shiftTimeZone(shiftTimeZone)
                         .keySerializer(KEY_SER)
                         .assigner(assigner)
                         .aggregate(wrapGenerated(aggsFunction), ACC_SER)
@@ -377,21 +379,21 @@ public void testEventTimeCumulativeWindows() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 1000L));
 
         testHarness.processWatermark(new Watermark(999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 1000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), localMills(1000L)));
         expectedOutput.add(new Watermark(999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(1999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 2000L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 2000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), localMills(2000L)));
+        expectedOutput.add(insertRecord("key2", 3L, 3L, localMills(0L), localMills(2000L)));
         expectedOutput.add(new Watermark(1999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(2999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 3000L));
-        expectedOutput.add(insertRecord("key2", 4L, 4L, 0L, 3000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), localMills(3000L)));
+        expectedOutput.add(insertRecord("key2", 4L, 4L, localMills(0L), localMills(3000L)));
         expectedOutput.add(new Watermark(2999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -408,7 +410,7 @@ public void testEventTimeCumulativeWindows() throws Exception {
         testHarness.open();
 
         testHarness.processWatermark(new Watermark(3999));
-        expectedOutput.add(insertRecord("key2", 1L, 1L, 3000L, 4000L));
+        expectedOutput.add(insertRecord("key2", 1L, 1L, localMills(3000L), localMills(4000L)));
         expectedOutput.add(new Watermark(3999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -417,7 +419,7 @@ public void testEventTimeCumulativeWindows() throws Exception {
         testHarness.processElement(insertRecord("key1", 2, 3500L));
 
         testHarness.processWatermark(new Watermark(4999));
-        expectedOutput.add(insertRecord("key2", 1L, 1L, 3000L, 5000L));
+        expectedOutput.add(insertRecord("key2", 1L, 1L, localMills(3000L), localMills(5000L)));
         expectedOutput.add(new Watermark(4999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -426,7 +428,7 @@ public void testEventTimeCumulativeWindows() throws Exception {
         testHarness.processElement(insertRecord("key1", 1, 4999L));
 
         testHarness.processWatermark(new Watermark(5999));
-        expectedOutput.add(insertRecord("key2", 1L, 1L, 3000L, 6000L));
+        expectedOutput.add(insertRecord("key2", 1L, 1L, localMills(3000L), localMills(6000L)));
         expectedOutput.add(new Watermark(5999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -449,16 +451,7 @@ public void testEventTimeCumulativeWindows() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeCumulativeWindowsInUTC() throws Exception {
-        testProcessingTimeCumulativeWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeCumulativeWindowsInShanghai() throws Exception {
-        testProcessingTimeCumulativeWindows(ZoneId.of("Asia/Shanghai"));
-    }
-
-    private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Exception {
+    public void testProcessingTimeCumulativeWindows() throws Exception {
         final SliceAssigner assigner =
                 SliceAssigners.cumulative(
                         -1, shiftTimeZone, Duration.ofDays(1), Duration.ofHours(8));
@@ -595,12 +588,12 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Ex
     @Test
     public void testEventTimeTumblingWindows() throws Exception {
         final SliceAssigner assigner =
-                SliceAssigners.tumbling(2, UTC_ZONE_ID, Duration.ofSeconds(3));
+                SliceAssigners.tumbling(2, shiftTimeZone, Duration.ofSeconds(3));
         final SumAndCountAggsFunction aggsFunction = new SumAndCountAggsFunction(assigner);
         SlicingWindowOperator operator =
                 SlicingWindowAggOperatorBuilder.builder()
                         .inputSerializer(INPUT_ROW_SER)
-                        .shiftTimeZone(UTC_ZONE_ID)
+                        .shiftTimeZone(shiftTimeZone)
                         .keySerializer(KEY_SER)
                         .assigner(assigner)
                         .aggregate(wrapGenerated(aggsFunction), ACC_SER)
@@ -649,8 +642,8 @@ public void testEventTimeTumblingWindows() throws Exception {
         testHarness.open();
 
         testHarness.processWatermark(new Watermark(2999));
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 3000L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 3000L));
+        expectedOutput.add(insertRecord("key1", 3L, 3L, localMills(0L), localMills(3000L)));
+        expectedOutput.add(insertRecord("key2", 3L, 3L, localMills(0L), localMills(3000L)));
         expectedOutput.add(new Watermark(2999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -672,7 +665,7 @@ public void testEventTimeTumblingWindows() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 2999L));
 
         testHarness.processWatermark(new Watermark(5999));
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 3000L, 6000L));
+        expectedOutput.add(insertRecord("key2", 2L, 2L, localMills(3000L), localMills(6000L)));
         expectedOutput.add(new Watermark(5999));
         ASSERTER.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -695,16 +688,7 @@ public void testEventTimeTumblingWindows() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeTumblingWindowsInUTC() throws Exception {
-        testProcessingTimeTumblingWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeTumblingWindowsInShanghai() throws Exception {
-        testProcessingTimeTumblingWindows(ZoneId.of("Asia/Shanghai"));
-    }
-
-    private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exception {
+    public void testProcessingTimeTumblingWindows() throws Exception {
 
         final SliceAssigner assigner =
                 SliceAssigners.tumbling(-1, shiftTimeZone, Duration.ofHours(5));
@@ -786,13 +770,14 @@ private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exce
     public void testInvalidWindows() {
         final SliceAssigner assigner =
                 SliceAssigners.hopping(
-                        2, UTC_ZONE_ID, Duration.ofSeconds(3), Duration.ofSeconds(1));
+                        2, shiftTimeZone, Duration.ofSeconds(3), Duration.ofSeconds(1));
         final SumAndCountAggsFunction aggsFunction = new SumAndCountAggsFunction(assigner);
 
         try {
             // hopping window without specifying count star index
             SlicingWindowAggOperatorBuilder.builder()
                     .inputSerializer(INPUT_ROW_SER)
+                    .shiftTimeZone(shiftTimeZone)
                     .keySerializer(KEY_SER)
                     .assigner(assigner)
                     .aggregate(wrapGenerated(aggsFunction), ACC_SER)
@@ -806,6 +791,11 @@ public void testInvalidWindows() {
         }
     }
 
+    /** Get the timestamp in mills by given epoch mills and timezone. */
+    private long localMills(long epochMills) {
+        return toUtcTimestampMills(epochMills, shiftTimeZone);
+    }
+
     private static OneInputStreamOperatorTestHarness createTestHarness(
             SlicingWindowOperator operator) throws Exception {
         return new KeyedOneInputStreamOperatorTestHarness<>(
@@ -966,4 +956,9 @@ private static long epochMills(ZoneId shiftTimeZone, String timestampStr) {
         ZoneOffset zoneOffset = shiftTimeZone.getRules().getOffset(localDateTime);
         return localDateTime.toInstant(zoneOffset).toEpochMilli();
     }
+
+    @Parameterized.Parameters(name = "TimeZone = {0}")
+    public static Collection runMode() {
+        return Arrays.asList(new Object[] {UTC_ZONE_ID}, new Object[] {SHANGHAI_ZONE_ID});
+    }
 }
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java
index 467352f9f031d..89e874831c9f7 100644
--- a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/operators/window/WindowOperatorTest.java
@@ -82,28 +82,35 @@
 @RunWith(Parameterized.class)
 public class WindowOperatorTest {
 
-    @Parameterized.Parameters(name = "isTableAggregate = {0}")
+    private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC");
+    private static final ZoneId SHANGHAI_ZONE_ID = ZoneId.of("Asia/Shanghai");
+    private final boolean isTableAggregate;
+    private final ZoneId shiftTimeZone;
+
+    @Parameterized.Parameters(name = "isTableAggregate = {0}, TimeZone = {1}")
     public static Collection runMode() {
-        return Arrays.asList(new Object[] {false}, new Object[] {true});
+        return Arrays.asList(
+                new Object[] {false, UTC_ZONE_ID},
+                new Object[] {true, UTC_ZONE_ID},
+                new Object[] {false, SHANGHAI_ZONE_ID},
+                new Object[] {true, SHANGHAI_ZONE_ID});
     }
 
-    private static final ZoneId UTC_ZONE_ID = ZoneId.of("UTC");
-    private static final ZoneId SHANGHAI_ZONE_ID = ZoneId.of("Asia/Shanghai");
+    public WindowOperatorTest(boolean isTableAggregate, ZoneId shiftTimeZone) {
+        this.isTableAggregate = isTableAggregate;
+        this.shiftTimeZone = shiftTimeZone;
+    }
 
-    private final boolean isTableAggregate;
     private static final SumAndCountAggTimeWindow sumAndCountAggTimeWindow =
             new SumAndCountAggTimeWindow();
     private static final SumAndCountTableAggTimeWindow sumAndCountTableAggTimeWindow =
             new SumAndCountTableAggTimeWindow();
     private static final SumAndCountAggCountWindow sumAndCountAggCountWindow =
             new SumAndCountAggCountWindow();
+
     private static final SumAndCountTableAggCountWindow sumAndCountTableAggCountWindow =
             new SumAndCountTableAggCountWindow();
 
-    public WindowOperatorTest(boolean isTableAggregate) {
-        this.isTableAggregate = isTableAggregate;
-    }
-
     private NamespaceAggsHandleFunctionBase getTimeWindowAggFunction() {
         return isTableAggregate ? sumAndCountTableAggTimeWindow : sumAndCountAggTimeWindow;
     }
@@ -159,7 +166,7 @@ public void testEventTimeSlidingWindows() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .sliding(Duration.ofSeconds(3), Duration.ofSeconds(1))
                         .withEventTime(2)
                         .aggregateAndBuild(
@@ -191,25 +198,65 @@ public void testEventTimeSlidingWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, -2000L, 1000L, 999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(-2000L),
+                                localMills(1000L),
+                                localMills(999L))));
         expectedOutput.add(new Watermark(999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(1999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, -1000L, 2000L, 1999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(-1000L),
+                                localMills(2000L),
+                                localMills(1999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 3L, 3L, -1000L, 2000L, 1999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                3L,
+                                3L,
+                                localMills(-1000L),
+                                localMills(2000L),
+                                localMills(1999L))));
         expectedOutput.add(new Watermark(1999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(2999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 3L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.add(new Watermark(2999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -226,21 +273,45 @@ public void testEventTimeSlidingWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(3999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 5L, 5L, 1000L, 4000L, 3999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                5L,
+                                5L,
+                                localMills(1000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.add(new Watermark(3999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(4999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 2L, 2L, 2000L, 5000L, 4999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                2L,
+                                2L,
+                                localMills(2000L),
+                                localMills(5000L),
+                                localMills(4999L))));
         expectedOutput.add(new Watermark(4999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(5999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 2L, 2L, 3000L, 6000L, 5999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                2L,
+                                2L,
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
         expectedOutput.add(new Watermark(5999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -261,16 +332,7 @@ public void testEventTimeSlidingWindows() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeSlidingWindowsinUTC() throws Throwable {
-        testProcessingTimeSlidingWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeSlidingWindowsinShanghai() throws Throwable {
-        testProcessingTimeSlidingWindows(SHANGHAI_ZONE_ID);
-    }
-
-    private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throwable {
+    public void testProcessingTimeSlidingWindows() throws Throwable {
         closeCalled.set(0);
 
         WindowOperator operator =
@@ -306,9 +368,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 1L,
                                 1L,
-                                toUtcTimestampMills(-2000L, shiftTimeZone),
-                                toUtcTimestampMills(1000L, shiftTimeZone),
-                                toUtcTimestampMills(999L, shiftTimeZone))));
+                                localMills(-2000L),
+                                localMills(1000L),
+                                localMills(999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -325,9 +387,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(-1000L, shiftTimeZone),
-                                toUtcTimestampMills(2000L, shiftTimeZone),
-                                toUtcTimestampMills(1999L, shiftTimeZone))));
+                                localMills(-1000L),
+                                localMills(2000L),
+                                localMills(1999L))));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
@@ -343,9 +405,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -353,9 +415,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -373,9 +435,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(1000L, shiftTimeZone),
-                                toUtcTimestampMills(4000L, shiftTimeZone),
-                                toUtcTimestampMills(3999L, shiftTimeZone))));
+                                localMills(1000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -383,9 +445,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key1",
                                 5L,
                                 5L,
-                                toUtcTimestampMills(1000L, shiftTimeZone),
-                                toUtcTimestampMills(4000L, shiftTimeZone),
-                                toUtcTimestampMills(3999L, shiftTimeZone))));
+                                localMills(1000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -393,9 +455,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key1",
                                 5L,
                                 5L,
-                                toUtcTimestampMills(2000L, shiftTimeZone),
-                                toUtcTimestampMills(5000L, shiftTimeZone),
-                                toUtcTimestampMills(4999L, shiftTimeZone))));
+                                localMills(2000L),
+                                localMills(5000L),
+                                localMills(4999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -403,9 +465,9 @@ private void testProcessingTimeSlidingWindows(ZoneId shiftTimeZone) throws Throw
                                 "key1",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(6000L, shiftTimeZone),
-                                toUtcTimestampMills(5999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -420,7 +482,7 @@ public void testEventTimeCumulativeWindows() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .cumulative(Duration.ofSeconds(3), Duration.ofSeconds(1))
                         .withEventTime(2)
                         .aggregateAndBuild(
@@ -452,25 +514,65 @@ public void testEventTimeCumulativeWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, 0L, 1000L, 999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(1000L),
+                                localMills(999L))));
         expectedOutput.add(new Watermark(999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(1999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, 0L, 2000L, 1999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(2000L),
+                                localMills(1999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 3L, 3L, 0L, 2000L, 1999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(2000L),
+                                localMills(1999L))));
         expectedOutput.add(new Watermark(1999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(2999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 4L, 4L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                4L,
+                                4L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.add(new Watermark(2999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -487,21 +589,45 @@ public void testEventTimeCumulativeWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(3999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 1L, 1L, 3000L, 4000L, 3999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                1L,
+                                1L,
+                                localMills(3000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.add(new Watermark(3999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(4999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 1L, 1L, 3000L, 5000L, 4999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                1L,
+                                1L,
+                                localMills(3000L),
+                                localMills(5000L),
+                                localMills(4999L))));
         expectedOutput.add(new Watermark(4999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(5999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 1L, 1L, 3000L, 6000L, 5999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                1L,
+                                1L,
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
         expectedOutput.add(new Watermark(5999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -526,7 +652,7 @@ public void testEventTimeCumulativeWindowsWithLateArrival() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .cumulative(Duration.ofSeconds(3), Duration.ofSeconds(1))
                         .withEventTime(2)
                         .withAllowedLateness(Duration.ofMillis(500))
@@ -548,13 +674,15 @@ public void testEventTimeCumulativeWindowsWithLateArrival() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 500L));
         testHarness.processWatermark(new Watermark(1500));
 
-        expectedOutput.add(insertRecord("key2", 1L, 1L, 0L, 1000L, 999L));
+        expectedOutput.add(
+                insertRecord("key2", 1L, 1L, localMills(0L), localMills(1000L), localMills(999L)));
         expectedOutput.add(new Watermark(1500));
 
         testHarness.processElement(insertRecord("key2", 1, 1300L));
         testHarness.processWatermark(new Watermark(2300));
 
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 0L, 2000L, 1999L));
+        expectedOutput.add(
+                insertRecord("key2", 2L, 2L, localMills(0L), localMills(2000L), localMills(1999L)));
         expectedOutput.add(new Watermark(2300));
 
         // this will not be dropped because window.maxTimestamp() + allowedLateness >
@@ -563,9 +691,14 @@ public void testEventTimeCumulativeWindowsWithLateArrival() throws Exception {
         testHarness.processWatermark(new Watermark(6000));
 
         // this is 1 and not 3 because the trigger fires and purges
-        expectedOutput.add(updateBeforeRecord("key2", 2L, 2L, 0L, 2000L, 1999L));
-        expectedOutput.add(updateAfterRecord("key2", 3L, 3L, 0L, 2000L, 1999L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 3000L, 2999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 2L, 2L, localMills(0L), localMills(2000L), localMills(1999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 3L, 3L, localMills(0L), localMills(2000L), localMills(1999L)));
+        expectedOutput.add(
+                insertRecord("key2", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
         expectedOutput.add(new Watermark(6000));
 
         // this will be dropped because window.maxTimestamp() + allowedLateness < currentWatermark
@@ -583,16 +716,7 @@ public void testEventTimeCumulativeWindowsWithLateArrival() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeCumulativeWindowsInUTC() throws Throwable {
-        testProcessingTimeCumulativeWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeCumulativeWindowsInShangHai() throws Throwable {
-        testProcessingTimeCumulativeWindows(SHANGHAI_ZONE_ID);
-    }
-
-    private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Throwable {
+    public void testProcessingTimeCumulativeWindows() throws Throwable {
         closeCalled.set(0);
 
         WindowOperator operator =
@@ -628,9 +752,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 1L,
                                 1L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(1000L, shiftTimeZone),
-                                toUtcTimestampMills(999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(1000L),
+                                localMills(999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -647,9 +771,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(2000L, shiftTimeZone),
-                                toUtcTimestampMills(1999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(2000L),
+                                localMills(1999L))));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
@@ -665,9 +789,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -675,9 +799,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -695,9 +819,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(4000L, shiftTimeZone),
-                                toUtcTimestampMills(3999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -705,9 +829,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 1L,
                                 1L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(4000L, shiftTimeZone),
-                                toUtcTimestampMills(3999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -715,9 +839,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(5000L, shiftTimeZone),
-                                toUtcTimestampMills(4999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(5000L),
+                                localMills(4999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -725,9 +849,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 1L,
                                 1L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(5000L, shiftTimeZone),
-                                toUtcTimestampMills(4999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(5000L),
+                                localMills(4999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -735,9 +859,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(6000L, shiftTimeZone),
-                                toUtcTimestampMills(5999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -745,9 +869,9 @@ private void testProcessingTimeCumulativeWindows(ZoneId shiftTimeZone) throws Th
                                 "key2",
                                 1L,
                                 1L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(6000L, shiftTimeZone),
-                                toUtcTimestampMills(5999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -763,7 +887,7 @@ public void testEventTimeTumblingWindows() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .tumble(Duration.ofSeconds(3))
                         .withEventTime(2)
                         .aggregateAndBuild(
@@ -814,9 +938,25 @@ public void testEventTimeTumblingWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(2999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 3L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                3L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.add(new Watermark(2999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -833,7 +973,15 @@ public void testEventTimeTumblingWindows() throws Exception {
 
         testHarness.processWatermark(new Watermark(5999));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 2L, 2L, 3000L, 6000L, 5999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                2L,
+                                2L,
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
         expectedOutput.add(new Watermark(5999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -862,7 +1010,7 @@ public void testEventTimeTumblingWindowsWithEarlyFiring() throws Exception {
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
                         .tumble(Duration.ofSeconds(3))
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .withEventTime(2)
                         .triggering(
                                 EventTimeTriggers.afterEndOfWindow()
@@ -900,15 +1048,19 @@ public void testEventTimeTumblingWindowsWithEarlyFiring() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 1000L));
 
         testHarness.setProcessingTime(1000);
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                insertRecord(
+                        "key2", 2L, 2L, localMills(3000L), localMills(6000L), localMills(5999L)));
         testHarness.processWatermark(new Watermark(999));
         expectedOutput.add(new Watermark(999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.setProcessingTime(1001);
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 3000L, 2999L));
+        expectedOutput.add(
+                insertRecord("key1", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
+        expectedOutput.add(
+                insertRecord("key2", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
 
         testHarness.processWatermark(new Watermark(1999));
         testHarness.setProcessingTime(2001);
@@ -939,8 +1091,12 @@ public void testEventTimeTumblingWindowsWithEarlyFiring() throws Exception {
         testHarness.processWatermark(new Watermark(3999));
         testHarness.setProcessingTime(4001);
         expectedOutput.add(new Watermark(3999));
-        expectedOutput.add(updateBeforeRecord("key2", 2L, 2L, 3000L, 6000L, 5999L));
-        expectedOutput.add(updateAfterRecord("key2", 3L, 3L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 2L, 2L, localMills(3000L), localMills(6000L), localMills(5999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 3L, 3L, localMills(3000L), localMills(6000L), localMills(5999L)));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
@@ -959,8 +1115,12 @@ public void testEventTimeTumblingWindowsWithEarlyFiring() throws Exception {
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(5999));
-        expectedOutput.add(updateBeforeRecord("key2", 3L, 3L, 3000L, 6000L, 5999L));
-        expectedOutput.add(updateAfterRecord("key2", 4L, 4L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 3L, 3L, localMills(3000L), localMills(6000L), localMills(5999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 4L, 4L, localMills(3000L), localMills(6000L), localMills(5999L)));
         expectedOutput.add(new Watermark(5999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -998,7 +1158,7 @@ public void testEventTimeTumblingWindowsWithEarlyAndLateFirings() throws Excepti
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .tumble(Duration.ofSeconds(3))
                         .withEventTime(2)
                         .triggering(
@@ -1038,15 +1198,20 @@ public void testEventTimeTumblingWindowsWithEarlyAndLateFirings() throws Excepti
         testHarness.processElement(insertRecord("key2", 1, 1000L));
 
         testHarness.setProcessingTime(1000);
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                insertRecord(
+                        "key2", 2L, 2L, localMills(3000L), localMills(6000L), localMills(5999L)));
         testHarness.processWatermark(new Watermark(999));
         expectedOutput.add(new Watermark(999));
+
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.setProcessingTime(1001);
-        expectedOutput.add(insertRecord("key1", 3L, 3L, 0L, 3000L, 2999L));
-        expectedOutput.add(insertRecord("key2", 3L, 3L, 0L, 3000L, 2999L));
+        expectedOutput.add(
+                insertRecord("key1", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
+        expectedOutput.add(
+                insertRecord("key2", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
 
         testHarness.processWatermark(new Watermark(1999));
         testHarness.setProcessingTime(2001);
@@ -1077,22 +1242,34 @@ public void testEventTimeTumblingWindowsWithEarlyAndLateFirings() throws Excepti
         testHarness.processWatermark(new Watermark(3999));
         testHarness.setProcessingTime(4001);
         expectedOutput.add(new Watermark(3999));
-        expectedOutput.add(updateBeforeRecord("key2", 2L, 2L, 3000L, 6000L, 5999L));
-        expectedOutput.add(updateAfterRecord("key2", 3L, 3L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 2L, 2L, localMills(3000L), localMills(6000L), localMills(5999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 3L, 3L, localMills(3000L), localMills(6000L), localMills(5999L)));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         // late arrival
         testHarness.processElement(insertRecord("key2", 1, 2001L));
-        expectedOutput.add(updateBeforeRecord("key2", 3L, 3L, 0L, 3000L, 2999L));
-        expectedOutput.add(updateAfterRecord("key2", 4L, 4L, 0L, 3000L, 2999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 4L, 4L, localMills(0L), localMills(3000L), localMills(2999L)));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         // late arrival
         testHarness.processElement(insertRecord("key1", 1, 2030L));
-        expectedOutput.add(updateBeforeRecord("key1", 3L, 3L, 0L, 3000L, 2999L));
-        expectedOutput.add(updateAfterRecord("key1", 4L, 4L, 0L, 3000L, 2999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key1", 3L, 3L, localMills(0L), localMills(3000L), localMills(2999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key1", 4L, 4L, localMills(0L), localMills(3000L), localMills(2999L)));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
@@ -1104,8 +1281,12 @@ public void testEventTimeTumblingWindowsWithEarlyAndLateFirings() throws Excepti
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
 
         testHarness.processWatermark(new Watermark(5999));
-        expectedOutput.add(updateBeforeRecord("key2", 3L, 3L, 3000L, 6000L, 5999L));
-        expectedOutput.add(updateAfterRecord("key2", 4L, 4L, 3000L, 6000L, 5999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 3L, 3L, localMills(3000L), localMills(6000L), localMills(5999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 4L, 4L, localMills(3000L), localMills(6000L), localMills(5999L)));
         expectedOutput.add(new Watermark(5999));
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -1136,17 +1317,8 @@ public void testEventTimeTumblingWindowsWithEarlyAndLateFirings() throws Excepti
     }
 
     @Test
-    public void testProcessingTimeTumblingWindowsInUTC() throws Exception {
-        testProcessingTimeTumblingWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeTumblingWindowsInShanghai() throws Exception {
-        testProcessingTimeTumblingWindows(SHANGHAI_ZONE_ID);
-    }
-
     @SuppressWarnings("unchecked")
-    private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exception {
+    public void testProcessingTimeTumblingWindows() throws Exception {
         closeCalled.set(0);
 
         WindowOperator operator =
@@ -1188,9 +1360,9 @@ private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exce
                                 "key2",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -1198,9 +1370,9 @@ private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exce
                                 "key1",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(0L, shiftTimeZone),
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(2999L, shiftTimeZone))));
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -1218,9 +1390,9 @@ private void testProcessingTimeTumblingWindows(ZoneId shiftTimeZone) throws Exce
                                 "key1",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(3000L, shiftTimeZone),
-                                toUtcTimestampMills(6000L, shiftTimeZone),
-                                toUtcTimestampMills(5999L, shiftTimeZone))));
+                                localMills(3000L),
+                                localMills(6000L),
+                                localMills(5999L))));
 
         assertEquals(0L, operator.getWatermarkLatency().getValue());
         assertor.assertOutputEqualsSorted(
@@ -1237,7 +1409,7 @@ public void testEventTimeSessionWindows() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .session(Duration.ofSeconds(3))
                         .withEventTime(2)
                         .aggregateAndBuild(
@@ -1284,12 +1456,36 @@ public void testEventTimeSessionWindows() throws Exception {
         testHarness.processWatermark(new Watermark(12000));
 
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 6L, 3L, 10L, 5500L, 5499L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                6L,
+                                3L,
+                                localMills(10L),
+                                localMills(5500L),
+                                localMills(5499L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 6L, 3L, 0L, 5500L, 5499L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                6L,
+                                3L,
+                                localMills(0L),
+                                localMills(5500L),
+                                localMills(5499L))));
 
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 20L, 4L, 5501L, 9050L, 9049L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                20L,
+                                4L,
+                                localMills(5501L),
+                                localMills(9050L),
+                                localMills(9049L))));
         expectedOutput.add(new Watermark(12000));
 
         // add a late data
@@ -1301,7 +1497,14 @@ public void testEventTimeSessionWindows() throws Exception {
 
         expectedOutput.addAll(
                 doubleRecord(
-                        isTableAggregate, insertRecord("key2", 30L, 2L, 15000L, 18000L, 17999L)));
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                30L,
+                                2L,
+                                localMills(15000L),
+                                localMills(18000L),
+                                localMills(17999L))));
         expectedOutput.add(new Watermark(17999));
 
         assertor.assertOutputEqualsSorted(
@@ -1318,22 +1521,12 @@ public void testEventTimeSessionWindows() throws Exception {
     }
 
     @Test
-    public void testProcessingTimeSessionWindowsInUTC() throws Throwable {
-        testProcessingTimeSessionWindows(UTC_ZONE_ID);
-    }
-
-    @Test
-    public void testProcessingTimeSessionWindowsInShanghai() throws Throwable {
-        testProcessingTimeSessionWindows(SHANGHAI_ZONE_ID);
-    }
-
-    private void testProcessingTimeSessionWindows(ZoneId shiftTimeZone) throws Throwable {
+    public void testProcessingTimeSessionWindows() throws Throwable {
         closeCalled.set(0);
 
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
                         .withShiftTimezone(shiftTimeZone)
                         .session(Duration.ofSeconds(3))
                         .withProcessingTime()
@@ -1373,9 +1566,9 @@ private void testProcessingTimeSessionWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(3L, shiftTimeZone),
-                                toUtcTimestampMills(4000L, shiftTimeZone),
-                                toUtcTimestampMills(3999L, shiftTimeZone))));
+                                localMills(3L),
+                                localMills(4000L),
+                                localMills(3999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -1395,9 +1588,9 @@ private void testProcessingTimeSessionWindows(ZoneId shiftTimeZone) throws Throw
                                 "key2",
                                 2L,
                                 2L,
-                                toUtcTimestampMills(5000L, shiftTimeZone),
-                                toUtcTimestampMills(8000L, shiftTimeZone),
-                                toUtcTimestampMills(7999L, shiftTimeZone))));
+                                localMills(5000L),
+                                localMills(8000L),
+                                localMills(7999L))));
         expectedOutput.addAll(
                 doubleRecord(
                         isTableAggregate,
@@ -1405,9 +1598,9 @@ private void testProcessingTimeSessionWindows(ZoneId shiftTimeZone) throws Throw
                                 "key1",
                                 3L,
                                 3L,
-                                toUtcTimestampMills(5000L, shiftTimeZone),
-                                toUtcTimestampMills(8000L, shiftTimeZone),
-                                toUtcTimestampMills(7999L, shiftTimeZone))));
+                                localMills(5000L),
+                                localMills(8000L),
+                                localMills(7999L))));
 
         assertor.assertOutputEqualsSorted(
                 "Output was not correct.", expectedOutput, testHarness.getOutput());
@@ -1430,7 +1623,7 @@ public void testPointSessions() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .assigner(new PointSessionWindowAssigner(3000))
                         .withEventTime(2)
                         .aggregateAndBuild(
@@ -1469,9 +1662,25 @@ public void testPointSessions() throws Exception {
         testHarness.processWatermark(new Watermark(12000));
 
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key1", 36L, 3L, 10L, 4000L, 3999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key1",
+                                36L,
+                                3L,
+                                localMills(10L),
+                                localMills(4000L),
+                                localMills(3999L))));
         expectedOutput.addAll(
-                doubleRecord(isTableAggregate, insertRecord("key2", 67L, 3L, 0L, 3000L, 2999L)));
+                doubleRecord(
+                        isTableAggregate,
+                        insertRecord(
+                                "key2",
+                                67L,
+                                3L,
+                                localMills(0L),
+                                localMills(3000L),
+                                localMills(2999L))));
         expectedOutput.add(new Watermark(12000));
 
         assertor.assertOutputEqualsSorted(
@@ -1488,7 +1697,7 @@ public void testLateness() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .tumble(Duration.ofSeconds(2))
                         .withEventTime(2)
                         .withAllowedLateness(Duration.ofMillis(500))
@@ -1515,7 +1724,8 @@ public void testLateness() throws Exception {
         testHarness.processElement(insertRecord("key2", 1, 1300L));
         testHarness.processWatermark(new Watermark(2300));
 
-        expectedOutput.add(insertRecord("key2", 2L, 2L, 0L, 2000L, 1999L));
+        expectedOutput.add(
+                insertRecord("key2", 2L, 2L, localMills(0L), localMills(2000L), localMills(1999L)));
         expectedOutput.add(new Watermark(2300));
 
         // this will not be dropped because window.maxTimestamp() + allowedLateness >
@@ -1524,8 +1734,12 @@ public void testLateness() throws Exception {
         testHarness.processWatermark(new Watermark(6000));
 
         // this is 1 and not 3 because the trigger fires and purges
-        expectedOutput.add(updateBeforeRecord("key2", 2L, 2L, 0L, 2000L, 1999L));
-        expectedOutput.add(updateAfterRecord("key2", 3L, 3L, 0L, 2000L, 1999L));
+        expectedOutput.add(
+                updateBeforeRecord(
+                        "key2", 2L, 2L, localMills(0L), localMills(2000L), localMills(1999L)));
+        expectedOutput.add(
+                updateAfterRecord(
+                        "key2", 3L, 3L, localMills(0L), localMills(2000L), localMills(1999L)));
         expectedOutput.add(new Watermark(6000));
 
         // this will be dropped because window.maxTimestamp() + allowedLateness < currentWatermark
@@ -1542,14 +1756,63 @@ public void testLateness() throws Exception {
         testHarness.close();
     }
 
+    @Test
+    public void testCleanupTimerWithEmptyReduceStateForTumblingWindows() throws Exception {
+        final int windowSize = 2;
+        final long lateness = 1;
+
+        WindowOperator operator =
+                WindowOperatorBuilder.builder()
+                        .withInputFields(inputFieldTypes)
+                        .withShiftTimezone(shiftTimeZone)
+                        .tumble(Duration.ofSeconds(windowSize))
+                        .withEventTime(2)
+                        .withAllowedLateness(Duration.ofMillis(lateness))
+                        .produceUpdates()
+                        .aggregateAndBuild(
+                                new SumAndCountAggTimeWindow(),
+                                equaliser,
+                                accTypes,
+                                aggResultTypes,
+                                windowTypes);
+
+        OneInputStreamOperatorTestHarness testHarness =
+                createTestHarness(operator);
+
+        testHarness.open();
+
+        ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>();
+
+        // normal element
+        testHarness.processElement(insertRecord("key2", 1, 1000L));
+        testHarness.processWatermark(new Watermark(1599));
+        testHarness.processWatermark(new Watermark(1999));
+        testHarness.processWatermark(new Watermark(2000));
+        testHarness.processWatermark(new Watermark(5000));
+
+        expected.add(new Watermark(1599));
+        expected.add(
+                insertRecord("key2", 1L, 1L, localMills(0L), localMills(2000L), localMills(1999L)));
+        expected.add(new Watermark(1999)); // here it fires and purges
+        expected.add(new Watermark(2000)); // here is the cleanup timer
+        expected.add(new Watermark(5000));
+
+        assertor.assertOutputEqualsSorted(
+                "Output was not correct.", expected, testHarness.getOutput());
+        testHarness.close();
+    }
+
     @Test
     public void testCleanupTimeOverflow() throws Exception {
+        if (!UTC_ZONE_ID.equals(shiftTimeZone)) {
+            return;
+        }
         long windowSize = 1000;
         long lateness = 2000;
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .tumble(Duration.ofMillis(windowSize))
                         .withEventTime(2)
                         .withAllowedLateness(Duration.ofMillis(lateness))
@@ -1607,53 +1870,11 @@ public void testCleanupTimeOverflow() throws Exception {
         testHarness.close();
     }
 
-    @Test
-    public void testCleanupTimerWithEmptyReduceStateForTumblingWindows() throws Exception {
-        final int windowSize = 2;
-        final long lateness = 1;
-
-        WindowOperator operator =
-                WindowOperatorBuilder.builder()
-                        .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
-                        .tumble(Duration.ofSeconds(windowSize))
-                        .withEventTime(2)
-                        .withAllowedLateness(Duration.ofMillis(lateness))
-                        .produceUpdates()
-                        .aggregateAndBuild(
-                                new SumAndCountAggTimeWindow(),
-                                equaliser,
-                                accTypes,
-                                aggResultTypes,
-                                windowTypes);
-
-        OneInputStreamOperatorTestHarness testHarness =
-                createTestHarness(operator);
-
-        testHarness.open();
-
-        ConcurrentLinkedQueue expected = new ConcurrentLinkedQueue<>();
-
-        // normal element
-        testHarness.processElement(insertRecord("key2", 1, 1000L));
-        testHarness.processWatermark(new Watermark(1599));
-        testHarness.processWatermark(new Watermark(1999));
-        testHarness.processWatermark(new Watermark(2000));
-        testHarness.processWatermark(new Watermark(5000));
-
-        expected.add(new Watermark(1599));
-        expected.add(insertRecord("key2", 1L, 1L, 0L, 2000L, 1999L));
-        expected.add(new Watermark(1999)); // here it fires and purges
-        expected.add(new Watermark(2000)); // here is the cleanup timer
-        expected.add(new Watermark(5000));
-
-        assertor.assertOutputEqualsSorted(
-                "Output was not correct.", expected, testHarness.getOutput());
-        testHarness.close();
-    }
-
     @Test
     public void testTumblingCountWindow() throws Exception {
+        if (!UTC_ZONE_ID.equals(shiftTimeZone)) {
+            return;
+        }
         closeCalled.set(0);
         final int windowSize = 3;
         LogicalType[] windowTypes = new LogicalType[] {new BigIntType()};
@@ -1661,7 +1882,7 @@ public void testTumblingCountWindow() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .countWindow(windowSize)
                         .aggregateAndBuild(
                                 getCountWindowAggFunction(),
@@ -1735,6 +1956,9 @@ public void testTumblingCountWindow() throws Exception {
 
     @Test
     public void testSlidingCountWindow() throws Exception {
+        if (!UTC_ZONE_ID.equals(shiftTimeZone)) {
+            return;
+        }
         closeCalled.set(0);
         final int windowSize = 5;
         final int windowSlide = 3;
@@ -1743,7 +1967,7 @@ public void testSlidingCountWindow() throws Exception {
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
-                        .withShiftTimezone(UTC_ZONE_ID)
+                        .withShiftTimezone(shiftTimeZone)
                         .countWindow(windowSize, windowSlide)
                         .aggregateAndBuild(
                                 getCountWindowAggFunction(),
@@ -1817,12 +2041,16 @@ public void testSlidingCountWindow() throws Exception {
 
     @Test
     public void testWindowCloseWithoutOpen() throws Exception {
+        if (!UTC_ZONE_ID.equals(shiftTimeZone)) {
+            return;
+        }
         final int windowSize = 3;
         LogicalType[] windowTypes = new LogicalType[] {new BigIntType()};
 
         WindowOperator operator =
                 WindowOperatorBuilder.builder()
                         .withInputFields(inputFieldTypes)
+                        .withShiftTimezone(shiftTimeZone)
                         .countWindow(windowSize)
                         .aggregate(
                                 new GeneratedNamespaceTableAggsHandleFunction<>(
@@ -1838,6 +2066,11 @@ public void testWindowCloseWithoutOpen() throws Exception {
 
     // --------------------------------------------------------------------------------
 
+    /** Get the timestamp in mills by given epoch mills and timezone. */
+    private long localMills(long epochMills) {
+        return toUtcTimestampMills(epochMills, shiftTimeZone);
+    }
+
     private static class PointSessionWindowAssigner extends SessionWindowAssigner {
         private static final long serialVersionUID = 1L;
 
diff --git a/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/TimeWindowUtilTest.java b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/TimeWindowUtilTest.java
new file mode 100644
index 0000000000000..feea72767b207
--- /dev/null
+++ b/flink-table/flink-table-runtime-blink/src/test/java/org/apache/flink/table/runtime/util/TimeWindowUtilTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.junit.Test;
+
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.TimeZone;
+
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMills;
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.toEpochMillsForTimer;
+import static org.apache.flink.table.runtime.util.TimeWindowUtil.toUtcTimestampMills;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link org.apache.flink.table.runtime.util.TimeWindowUtil}. */
+public class TimeWindowUtilTest {
+
+    private static final ZoneId UTC_ZONE_ID = TimeZone.getTimeZone("UTC").toZoneId();
+
+    @Test
+    public void testShiftedTimeZone() {
+        ZoneId zoneId = ZoneId.of("Asia/Shanghai");
+        assertEquals(-28799000L, toEpochMillsForTimer(utcMills("1970-01-01T00:00:01"), zoneId));
+        assertEquals(-1L, toEpochMillsForTimer(utcMills("1970-01-01T07:59:59.999"), zoneId));
+        assertEquals(1000L, toEpochMillsForTimer(utcMills("1970-01-01T08:00:01"), zoneId));
+        assertEquals(1L, toEpochMillsForTimer(utcMills("1970-01-01T08:00:00.001"), zoneId));
+
+        assertEquals(-28799000L, toEpochMills(utcMills("1970-01-01T00:00:01"), zoneId));
+        assertEquals(-1L, toEpochMills(utcMills("1970-01-01T07:59:59.999"), zoneId));
+        assertEquals(1000L, toEpochMills(utcMills("1970-01-01T08:00:01"), zoneId));
+        assertEquals(1L, toEpochMills(utcMills("1970-01-01T08:00:00.001"), zoneId));
+    }
+
+    @Test
+    public void testDaylightSaving() {
+        ZoneId zoneId = ZoneId.of("America/Los_Angeles");
+        /*
+         * The DaylightTime in Los_Angele start at time 2021-03-14 02:00:00
+         * 
+         *  2021-03-14 00:00:00 -> epoch1 = 1615708800000L;
+         *  2021-03-14 01:00:00 -> epoch2 = 1615712400000L;
+         *  2021-03-14 03:00:00 -> epoch3 = 1615716000000L;  skip one hour (2021-03-14 02:00:00)
+         *  2021-03-14 04:00:00 -> epoch4 = 1615719600000L;
+         */
+        assertEquals(1615708800000L, toEpochMillsForTimer(utcMills("2021-03-14T00:00:00"), zoneId));
+        assertEquals(1615712400000L, toEpochMillsForTimer(utcMills("2021-03-14T01:00:00"), zoneId));
+        assertEquals(1615716000000L, toEpochMillsForTimer(utcMills("2021-03-14T02:00:00"), zoneId));
+        assertEquals(1615716000000L, toEpochMillsForTimer(utcMills("2021-03-14T02:30:00"), zoneId));
+        assertEquals(1615716000000L, toEpochMillsForTimer(utcMills("2021-03-14T02:59:59"), zoneId));
+        assertEquals(1615716000000L, toEpochMillsForTimer(utcMills("2021-03-14T03:00:00"), zoneId));
+        assertEquals(1615717800000L, toEpochMillsForTimer(utcMills("2021-03-14T03:30:00"), zoneId));
+        assertEquals(1615719599000L, toEpochMillsForTimer(utcMills("2021-03-14T03:59:59"), zoneId));
+
+        assertEquals(1615708800000L, toEpochMills(utcMills("2021-03-14T00:00:00"), zoneId));
+        assertEquals(1615712400000L, toEpochMills(utcMills("2021-03-14T01:00:00"), zoneId));
+        assertEquals(1615716000000L, toEpochMills(utcMills("2021-03-14T02:00:00"), zoneId));
+        assertEquals(1615717800000L, toEpochMills(utcMills("2021-03-14T02:30:00"), zoneId));
+        assertEquals(1615719599000L, toEpochMills(utcMills("2021-03-14T02:59:59"), zoneId));
+        assertEquals(1615717800000L, toEpochMills(utcMills("2021-03-14T03:30:00"), zoneId));
+        assertEquals(1615716000000L, toEpochMills(utcMills("2021-03-14T03:00:00"), zoneId));
+
+        /*
+         * The DaylightTime in Los_Angele end at time 2021-11-07 01:00:00
+         * 
+         *  2021-11-07 00:00:00 -> epoch0 = 1636268400000L;  2021-11-07 00:00:00
+         *  2021-11-07 01:00:00 -> epoch1 = 1636272000000L;  the first local timestamp 2021-11-07 01:00:00
+         *  2021-11-07 01:00:00 -> epoch2 = 1636275600000L;  back to local timestamp 2021-11-07 01:00:00
+         *  2021-11-07 02:00:00 -> epoch3 = 1636279200000L;  2021-11-07 02:00:00
+         */
+
+        assertEquals(utcMills("2021-11-07T01:00:00"), toUtcTimestampMills(1636272000000L, zoneId));
+        assertEquals(utcMills("2021-11-07T01:00:00"), toUtcTimestampMills(1636275600000L, zoneId));
+        assertEquals(utcMills("2021-11-07T01:00:01"), toUtcTimestampMills(1636272001000L, zoneId));
+        assertEquals(utcMills("2021-11-07T01:59:59"), toUtcTimestampMills(1636275599000L, zoneId));
+
+        assertEquals(1636268400000L, toEpochMillsForTimer(utcMills("2021-11-07T00:00:00"), zoneId));
+        assertEquals(1636275600000L, toEpochMillsForTimer(utcMills("2021-11-07T01:00:00"), zoneId));
+        assertEquals(1636279200000L, toEpochMillsForTimer(utcMills("2021-11-07T02:00:00"), zoneId));
+        assertEquals(1636268401000L, toEpochMillsForTimer(utcMills("2021-11-07T00:00:01"), zoneId));
+        assertEquals(1636279199000L, toEpochMillsForTimer(utcMills("2021-11-07T01:59:59"), zoneId));
+        assertEquals(1636279201000L, toEpochMillsForTimer(utcMills("2021-11-07T02:00:01"), zoneId));
+
+        assertEquals(1636268400000L, toEpochMills(utcMills("2021-11-07T00:00:00"), zoneId));
+        assertEquals(1636272000000L, toEpochMills(utcMills("2021-11-07T01:00:00"), zoneId));
+        assertEquals(1636279200000L, toEpochMills(utcMills("2021-11-07T02:00:00"), zoneId));
+        assertEquals(1636268401000L, toEpochMills(utcMills("2021-11-07T00:00:01"), zoneId));
+        assertEquals(1636275599000L, toEpochMills(utcMills("2021-11-07T01:59:59"), zoneId));
+        assertEquals(1636279201000L, toEpochMills(utcMills("2021-11-07T02:00:01"), zoneId));
+    }
+
+    @Test
+    public void testMaxWatermark() {
+        ZoneId zoneId = ZoneId.of("Asia/Shanghai");
+        assertEquals(Long.MAX_VALUE, toUtcTimestampMills(Long.MAX_VALUE, zoneId));
+        assertEquals(Long.MAX_VALUE, toEpochMillsForTimer(Long.MAX_VALUE, zoneId));
+        assertEquals(Long.MAX_VALUE, toEpochMills(Long.MAX_VALUE, zoneId));
+    }
+
+    private static long utcMills(String utcDateTime) {
+        return LocalDateTime.parse(utcDateTime).atZone(UTC_ZONE_ID).toInstant().toEpochMilli();
+    }
+}