twalthr commented on a change in pull request #17759:
URL: https://github.com/apache/flink/pull/17759#discussion_r751219368



##########
File path: 
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/sink/StreamRecordTimestampInserter.java
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.operators.TableStreamOperator;
+
+/**
+ * Operator which sets the timestamp on the StreamRecord from the 
corresponding column of each row.
+ */
+@Internal
+public class StreamRecordTimestampInserter extends TableStreamOperator<RowData>
+        implements OneInputStreamOperator<RowData, RowData> {
+
+    private final int rowtimeIndex;
+    private final int precision;
+
+    public StreamRecordTimestampInserter(int rowtimeIndex, int precision) {
+        this.rowtimeIndex = rowtimeIndex;
+        this.precision = precision;
+    }
+
+    public StreamRecordTimestampInserter(int rowtimeIndex) {
+        this(rowtimeIndex, 3);
+    }
+
+    @Override
+    public void processElement(StreamRecord<RowData> element) throws Exception 
{
+        final RowData rowData = element.getValue();
+        if (rowtimeIndex != -1) {

Review comment:
       should not be -1 at this point?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.factories.TableFactoryHarness;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
+import org.apache.flink.types.Row;
+
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP_LTZ;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link CommonExecSink}. */
+@RunWith(Parameterized.class)
+public class CommonExecSinkITCase extends AbstractTestBase {
+
+    private StreamExecutionEnvironment env;
+
+    enum ObjectReuse {
+        ENABLED,
+        DISABLED
+    }
+
+    @Parameterized.Parameters(name = "objectReuse = {0}")
+    public static ObjectReuse[] objectReuse() {
+        return ObjectReuse.values();
+    }
+
+    @Parameterized.Parameter public ObjectReuse objectReuse;
+
+    @Before
+    public void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(4);
+        if (objectReuse == ObjectReuse.ENABLED) {
+            env.getConfig().enableObjectReuse();
+        } else if (objectReuse == ObjectReuse.DISABLED) {
+            env.getConfig().disableObjectReuse();
+        }
+    }
+
+    @Rule public final SharedObjects sharedObjects = SharedObjects.create();
+
+    @Test
+    public void testStreamRecordTimestampInserterSinkRuntimeProvider()
+            throws ExecutionException, InterruptedException {
+        final StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env);
+        final SharedReference<List<Long>> timestamps = sharedObjects.add(new 
ArrayList<>());
+        final Schema schema = schemaForTimeOperatorTesting(true);
+        final List<Row> rows =
+                Arrays.asList(
+                        Row.of(1, "foo", 
Instant.parse("2020-11-10T12:34:56.123Z")),
+                        Row.of(2, "foo", 
Instant.parse("2020-11-10T11:34:56.789Z")),
+                        Row.of(3, "foo", 
Instant.parse("2020-11-11T10:11:22.777Z")),
+                        Row.of(4, "foo", 
Instant.parse("2020-11-11T10:11:23.888Z")));
+
+        final TableDescriptor sourceDescriptor =
+                TableFactoryHarness.newBuilder()
+                        .schema(schema)
+                        .source(
+                                new TimestampTestSource(
+                                        rows, ROW(INT(), STRING(), 
TIMESTAMP_LTZ(3))))
+                        .sink(
+                                new TableFactoryHarness.SinkBase() {
+                                    @Override
+                                    public DynamicTableSink.SinkRuntimeProvider
+                                            getSinkRuntimeProvider(
+                                                    DynamicTableSink.Context 
context) {
+                                        return SinkProvider.of(
+                                                TestSink.newBuilder()
+                                                        .setWriter(new 
TestWriter(timestamps))
+                                                        
.setCommittableSerializer(
+                                                                
TestSink.StringCommittableSerializer
+                                                                        
.INSTANCE)
+                                                        .build());
+                                    }
+                                })
+                        .build();
+        tableEnv.createTable("T1", sourceDescriptor);
+        String sqlStmt = "INSERT INTO T1 SELECT * FROM T1";
+        assertPlan(tableEnv, sqlStmt, true);
+        tableEnv.executeSql(sqlStmt).await();
+        assertTimestampResults(timestamps, rows);
+    }
+
+    @Test
+    public void testStreamRecordTimestampInserterDataStreamSinkProvider()
+            throws ExecutionException, InterruptedException {
+        final StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env);
+        final SharedReference<List<Long>> timestamps = sharedObjects.add(new 
ArrayList<>());
+        final Schema schema = schemaForTimeOperatorTesting(true);
+        final List<Row> rows =
+                Arrays.asList(
+                        Row.of(1, "foo", 
Instant.parse("2020-11-10T11:34:56.123Z")),
+                        Row.of(2, "foo", 
Instant.parse("2020-11-10T12:34:56.789Z")),
+                        Row.of(3, "foo", 
Instant.parse("2020-11-11T10:11:22.777Z")),
+                        Row.of(4, "foo", 
Instant.parse("2020-11-11T10:11:23.888Z")));
+
+        final TableDescriptor sourceDescriptor =
+                TableFactoryHarness.newBuilder()
+                        .schema(schema)
+                        .source(
+                                new TimestampTestSource(
+                                        rows, ROW(INT(), STRING(), 
TIMESTAMP_LTZ(3))))

Review comment:
       use `schema.toPhysicalRowDataType`? this would also avoid the mismatch 
that currently exists in `TIMESTAMP` vs `TIMESTAMP_LTZ`

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
##########
@@ -259,12 +258,10 @@ private void checkOrderKeys(RowType inputRowType) {
             Transformation<RowData> transform =
                     new OneInputTransformation<>(
                             inputTransform,
-                            String.format("rowtime field: (%s)", 
timeOrderFieldIdx),
-                            new ProcessOperator<>(
-                                    new RowtimeProcessFunction(

Review comment:
       delete `RowtimeProcessFunction`?

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.factories.TableFactoryHarness;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
+import org.apache.flink.types.Row;
+
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP_LTZ;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link CommonExecSink}. */
+@RunWith(Parameterized.class)
+public class CommonExecSinkITCase extends AbstractTestBase {
+
+    private StreamExecutionEnvironment env;
+
+    enum ObjectReuse {
+        ENABLED,
+        DISABLED
+    }
+
+    @Parameterized.Parameters(name = "objectReuse = {0}")
+    public static ObjectReuse[] objectReuse() {
+        return ObjectReuse.values();
+    }
+
+    @Parameterized.Parameter public ObjectReuse objectReuse;
+
+    @Before
+    public void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);

Review comment:
       remove and use default one

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.factories.TableFactoryHarness;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
+import org.apache.flink.types.Row;
+
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP_LTZ;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link CommonExecSink}. */
+@RunWith(Parameterized.class)
+public class CommonExecSinkITCase extends AbstractTestBase {
+
+    private StreamExecutionEnvironment env;
+
+    enum ObjectReuse {
+        ENABLED,
+        DISABLED
+    }
+
+    @Parameterized.Parameters(name = "objectReuse = {0}")
+    public static ObjectReuse[] objectReuse() {
+        return ObjectReuse.values();
+    }
+
+    @Parameterized.Parameter public ObjectReuse objectReuse;
+
+    @Before
+    public void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRuntimeMode(RuntimeExecutionMode.STREAMING);
+        env.setParallelism(4);
+        if (objectReuse == ObjectReuse.ENABLED) {
+            env.getConfig().enableObjectReuse();
+        } else if (objectReuse == ObjectReuse.DISABLED) {
+            env.getConfig().disableObjectReuse();
+        }
+    }
+
+    @Rule public final SharedObjects sharedObjects = SharedObjects.create();
+
+    @Test
+    public void testStreamRecordTimestampInserterSinkRuntimeProvider()
+            throws ExecutionException, InterruptedException {
+        final StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env);
+        final SharedReference<List<Long>> timestamps = sharedObjects.add(new 
ArrayList<>());
+        final Schema schema = schemaForTimeOperatorTesting(true);
+        final List<Row> rows =
+                Arrays.asList(
+                        Row.of(1, "foo", 
Instant.parse("2020-11-10T12:34:56.123Z")),
+                        Row.of(2, "foo", 
Instant.parse("2020-11-10T11:34:56.789Z")),
+                        Row.of(3, "foo", 
Instant.parse("2020-11-11T10:11:22.777Z")),
+                        Row.of(4, "foo", 
Instant.parse("2020-11-11T10:11:23.888Z")));
+
+        final TableDescriptor sourceDescriptor =
+                TableFactoryHarness.newBuilder()
+                        .schema(schema)
+                        .source(
+                                new TimestampTestSource(
+                                        rows, ROW(INT(), STRING(), 
TIMESTAMP_LTZ(3))))
+                        .sink(
+                                new TableFactoryHarness.SinkBase() {
+                                    @Override
+                                    public DynamicTableSink.SinkRuntimeProvider
+                                            getSinkRuntimeProvider(
+                                                    DynamicTableSink.Context 
context) {
+                                        return SinkProvider.of(
+                                                TestSink.newBuilder()
+                                                        .setWriter(new 
TestWriter(timestamps))
+                                                        
.setCommittableSerializer(
+                                                                
TestSink.StringCommittableSerializer
+                                                                        
.INSTANCE)
+                                                        .build());
+                                    }
+                                })
+                        .build();
+        tableEnv.createTable("T1", sourceDescriptor);
+        String sqlStmt = "INSERT INTO T1 SELECT * FROM T1";
+        assertPlan(tableEnv, sqlStmt, true);
+        tableEnv.executeSql(sqlStmt).await();
+        assertTimestampResults(timestamps, rows);
+    }
+
+    @Test
+    public void testStreamRecordTimestampInserterDataStreamSinkProvider()
+            throws ExecutionException, InterruptedException {
+        final StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env);
+        final SharedReference<List<Long>> timestamps = sharedObjects.add(new 
ArrayList<>());
+        final Schema schema = schemaForTimeOperatorTesting(true);
+        final List<Row> rows =
+                Arrays.asList(
+                        Row.of(1, "foo", 
Instant.parse("2020-11-10T11:34:56.123Z")),
+                        Row.of(2, "foo", 
Instant.parse("2020-11-10T12:34:56.789Z")),
+                        Row.of(3, "foo", 
Instant.parse("2020-11-11T10:11:22.777Z")),
+                        Row.of(4, "foo", 
Instant.parse("2020-11-11T10:11:23.888Z")));
+
+        final TableDescriptor sourceDescriptor =
+                TableFactoryHarness.newBuilder()
+                        .schema(schema)
+                        .source(
+                                new TimestampTestSource(
+                                        rows, ROW(INT(), STRING(), 
TIMESTAMP_LTZ(3))))
+                        .sink(
+                                new TableFactoryHarness.SinkBase() {
+                                    @Override
+                                    public DataStreamSinkProvider 
getSinkRuntimeProvider(
+                                            DynamicTableSink.Context context) {
+                                        return dataStream ->
+                                                dataStream.addSink(
+                                                        new 
SinkFunction<RowData>() {
+                                                            @Override
+                                                            public void invoke(
+                                                                    RowData 
value,
+                                                                    Context 
context) {
+                                                                addTimestamp(
+                                                                        
timestamps,
+                                                                        
context.timestamp());
+                                                            }
+                                                        });
+                                    }
+                                })
+                        .build();
+        tableEnv.createTable("T1", sourceDescriptor);
+        String sqlStmt = "INSERT INTO T1 SELECT * FROM T1";
+        assertPlan(tableEnv, sqlStmt, true);
+        tableEnv.executeSql(sqlStmt).await();
+        Collections.sort(timestamps.get());
+        assertTimestampResults(timestamps, rows);
+    }
+
+    @Test
+    public void testStreamRecordTimestampInserterNotApplied() {
+        final StreamTableEnvironment tableEnv = 
StreamTableEnvironment.create(env);
+        final SharedReference<List<Long>> timestamps = sharedObjects.add(new 
ArrayList<>());
+        final Schema schema = schemaForTimeOperatorTesting(false);
+        final List<Row> rows =
+                Arrays.asList(
+                        Row.of(1, "foo", 
Instant.parse("2020-11-10T11:34:56.123Z")),
+                        Row.of(2, "foo", 
Instant.parse("2020-11-10T12:34:56.789Z")),
+                        Row.of(3, "foo", 
Instant.parse("2020-11-11T10:11:22.777Z")),
+                        Row.of(4, "foo", 
Instant.parse("2020-11-11T10:11:23.888Z")));
+
+        final TableDescriptor sourceDescriptor =
+                TableFactoryHarness.newBuilder()
+                        .schema(schema)
+                        .source(
+                                new TimestampTestSource(
+                                        rows, ROW(INT(), STRING(), 
TIMESTAMP_LTZ(3))))
+                        .sink(
+                                new TableFactoryHarness.SinkBase() {
+                                    @Override
+                                    public DynamicTableSink.SinkRuntimeProvider
+                                            getSinkRuntimeProvider(
+                                                    DynamicTableSink.Context 
context) {
+                                        return SinkProvider.of(
+                                                TestSink.newBuilder()
+                                                        .setWriter(new 
TestWriter(timestamps))
+                                                        
.setCommittableSerializer(
+                                                                
TestSink.StringCommittableSerializer
+                                                                        
.INSTANCE)
+                                                        .build());
+                                    }
+                                })
+                        .build();
+        tableEnv.createTable("T1", sourceDescriptor);
+        assertPlan(tableEnv, "INSERT INTO T1 SELECT * FROM T1", false);
+    }
+
+    private static void addTimestamp(SharedReference<List<Long>> timestamps, 
Long timestamp) {
+        timestamps.applySync(l -> l.add(timestamp));
+    }
+
+    private static void assertPlan(
+            StreamTableEnvironment tableEnv,
+            String sql,
+            boolean containsStreamRecordTimestampInserter) {
+        Matcher<String> matcher = 
containsString("StreamRecordTimestampInserter(rowtime field: 2");
+        if (!containsStreamRecordTimestampInserter) {
+            matcher = not(matcher);
+        }
+        assertThat(tableEnv.explainSql(sql, 
ExplainDetail.JSON_EXECUTION_PLAN), matcher);
+    }
+
+    private static Schema schemaForTimeOperatorTesting(boolean withWatermark) {
+        Schema.Builder builder =
+                Schema.newBuilder()
+                        .column("a", "INT")
+                        .column("b", "STRING")
+                        .column("ts", "TIMESTAMP(3)");
+        if (withWatermark) {
+            builder.watermark("ts", "ts");
+        }
+        return builder.build();
+    }
+
+    private static void assertTimestampResults(
+            SharedReference<List<Long>> timestamps, List<Row> rows) {
+        assertEquals(rows.size(), timestamps.get().size());
+        for (int i = 0; i < rows.size(); i++) {
+            assertEquals(rows.get(i).getField(2), 
Instant.ofEpochMilli(timestamps.get().get(i)));
+        }
+    }
+
+    private static class TimestampTestSource extends 
TableFactoryHarness.ScanSourceBase {
+
+        private final List<Row> rows;
+        private final DataType rowDataType;

Review comment:
       call this `producedDataType` for consistency with other connectors

##########
File path: 
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java
##########
@@ -0,0 +1,315 @@
+/*
+ * 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.nodes.exec.common;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableDescriptor;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.connector.sink.DataStreamSinkProvider;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceFunctionProvider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.factories.TableFactoryHarness;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.testutils.junit.SharedObjects;
+import org.apache.flink.testutils.junit.SharedReference;
+import org.apache.flink.types.Row;
+
+import org.hamcrest.Matcher;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.ROW;
+import static org.apache.flink.table.api.DataTypes.STRING;
+import static org.apache.flink.table.api.DataTypes.TIMESTAMP_LTZ;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link CommonExecSink}. */
+@RunWith(Parameterized.class)
+public class CommonExecSinkITCase extends AbstractTestBase {
+
+    private StreamExecutionEnvironment env;
+
+    enum ObjectReuse {
+        ENABLED,
+        DISABLED
+    }
+
+    @Parameterized.Parameters(name = "objectReuse = {0}")
+    public static ObjectReuse[] objectReuse() {

Review comment:
       simplify the test by remove the Object reuse setting and choose the 
default one?

##########
File path: 
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
##########
@@ -351,6 +356,21 @@ private int deriveSinkParallelism(
                 sinkParallelism);
     }
 
+    private Transformation<RowData> applyTimestampTransformation(

Review comment:
       nit: call this `applyRowtimeTransformation` because "timestamp" sounds 
very generic given all the different types of timestamps we have in the planner




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to