You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "bvarghese1 (via GitHub)" <gi...@apache.org> on 2024/01/03 04:02:18 UTC

[PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

bvarghese1 opened a new pull request, #24020:
URL: https://github.com/apache/flink/pull/24020

   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   *Add restore tests for TableSourceScan node*
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
   - Added restore tests for TableSourceScan node which verifies the generated compiled plan with the saved compiled plan.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (no)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (no)
     - The serializers: (no)
     - The runtime per-record code paths (performance sensitive): (no)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
     - The S3 file system connector:  (no)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (no)
     - If yes, how is the feature documented? (not applicable)
   


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "bvarghese1 (via GitHub)" <gi...@apache.org>.
bvarghese1 commented on PR #24020:
URL: https://github.com/apache/flink/pull/24020#issuecomment-1883622718

   > From very quick skimming... I looked at `SourceAbilitySpec`, and it looks like this test ought to help cover the `JsonSubTypes` there.
   > 
   > I think I see the following readily:
   > 
   > ```
   > ProjectPushDownSpec
   > FilterPushDownSpec
   > LimitPushDownSpec
   > PartitionPushDownSpec
   > ReadingMetadataSpec
   > WatermarkPushDownSpec
   > ```
   > 
   > Is there coverage for the other two?
   > 
   > ```
   > SourceWatermarkSpec
   > AggregatePushDownSpec
   > ```
   
   Added coverage for `SourceWatermarkSpec`.
   `AggregatePushDownSpec` is only supported in BATCH mode - https://github.com/apache/flink/blob/master/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java#L106


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "jnh5y (via GitHub)" <gi...@apache.org>.
jnh5y commented on PR #24020:
URL: https://github.com/apache/flink/pull/24020#issuecomment-1877525393

   From very quick skimming... I looked at `SourceAbilitySpec`, and it looks like this test ought to help cover the `JsonSubTypes` there.  
   
   I think I see the following readily:
   ```
   ProjectPushDownSpec
   FilterPushDownSpec
   LimitPushDownSpec
   PartitionPushDownSpec
   ReadingMetadataSpec
   WatermarkPushDownSpec
   ```
   
   Is there coverage for the other two?
   ```
   SourceWatermarkSpec
   AggregatePushDownSpec
   ```


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "flinkbot (via GitHub)" <gi...@apache.org>.
flinkbot commented on PR #24020:
URL: https://github.com/apache/flink/pull/24020#issuecomment-1874824806

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "32e25c7d0a6c58dc23ae28437f7159bf449bdce1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32e25c7d0a6c58dc23ae28437f7159bf449bdce1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 32e25c7d0a6c58dc23ae28437f7159bf449bdce1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "dawidwys (via GitHub)" <gi...@apache.org>.
dawidwys commented on code in PR #24020:
URL: https://github.com/apache/flink/pull/24020#discussion_r1447295113


##########
flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java:
##########
@@ -198,6 +198,18 @@ public SqlTestStep getRunSqlTestStep() {
         return (SqlTestStep) sqlSteps.get(0);
     }
 
+    /**
+     * Convenience method to avoid boilerplate code. It assumes only one statement set is tested.
+     */
+    public StatementSetTestStep getRunStatementSetTestStep() {
+        List<TestStep> statementSetSteps =
+                runSteps.stream()
+                        .filter(s -> s.getKind() == TestKind.STATEMENT_SET)
+                        .collect(Collectors.toList());
+
+        return (StatementSetTestStep) statementSetSteps.get(0);

Review Comment:
   If we make that assumption, can we verify it similarly as we do for `getRunSqlTestStep`



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceScanTestPrograms.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.stream;
+
+import org.apache.flink.table.test.program.SinkTestStep;
+import org.apache.flink.table.test.program.SourceTestStep;
+import org.apache.flink.table.test.program.TableTestProgram;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+
+/** {@link TableTestProgram} definitions for testing {@link StreamExecTableSourceScan}. */
+public class TableSourceScanTestPrograms {
+
+    static final Row[] BEFORE_DATA = {
+        Row.of(1, 1L, "hi", DateTimeUtils.toLocalDateTime(1586937601000L)),
+        Row.of(2, 2L, "hello", DateTimeUtils.toLocalDateTime(1586937602000L)),
+        Row.of(3, 2L, "hello world", DateTimeUtils.toLocalDateTime(1586937603000L))
+    };
+
+    static final Row[] AFTER_DATA = {
+        Row.of(4, 4L, "foo", DateTimeUtils.toLocalDateTime(1586937614000L)),
+        Row.of(5, 2L, "foo bar", DateTimeUtils.toLocalDateTime(1586937615000L)),
+    };
+
+    static final TableTestProgram PROJECT_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-project-pushdown",
+                            "validates table source scan with project pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")
+                    .build();
+
+    static final TableTestProgram PROJECT_PUSHDOWN_DISABLED =
+            TableTestProgram.of(
+                            "table-source-scan-project-push-down-disabled",
+                            "validates table source scan with project pushdown disabled")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema(
+                                            "a INT",
+                                            "b BIGINT",
+                                            "c VARCHAR",
+                                            "ts TIMESTAMP(3) METADATA")
+                                    .addOption("readable-metadata", "ts:TIMESTAMP(3)")
+                                    .addOption("enable-projection-push-down", "false")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")
+                    .build();
+
+    static final TableTestProgram FILTER_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-filter-pushdown",
+                            "validates table source scan with filter pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .addOption("filterable-fields", "a")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[2, 2, hello]", "+I[3, 2, hello world]")
+                                    .consumedAfterRestore("+I[4, 4, foo]", "+I[5, 2, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT * FROM source_t WHERE a > 1")
+                    .build();
+
+    static final TableTestProgram LIMIT_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-limit-pushdown",
+                            "validates table source scan with limit pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore("+I[1, 1, hi]", "+I[2, 2, hello]")
+                                    .consumedAfterRestore(new String[] {})
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, b, c FROM source_t LIMIT 2")
+                    .build();
+
+    static final TableTestProgram PARTITION_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-partition-pushdown",
+                            "validates table source scan with partition pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .addPartitionKeys("b")
+                                    .addOption("partition-list", "b:1,b:2")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[2, 2, hello]", "+I[3, 2, hello world]")
+                                    .consumedAfterRestore("+I[5, 2, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, b, c FROM source_t WHERE b = 2")
+                    .build();
+
+    static final TableTestProgram READING_METADATA =
+            TableTestProgram.of(
+                            "table-source-scan-reading-metadata",
+                            "validates table source scan by reading metadata")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema(
+                                            "a INT",
+                                            "b BIGINT",
+                                            "c VARCHAR",
+                                            "d TIMESTAMP(3) METADATA")
+                                    .addOption("readable-metadata", "d:TIMESTAMP(3)")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")

Review Comment:
   This will not read the metadata, will it?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "bvarghese1 (via GitHub)" <gi...@apache.org>.
bvarghese1 commented on code in PR #24020:
URL: https://github.com/apache/flink/pull/24020#discussion_r1447722550


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceScanTestPrograms.java:
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.stream;
+
+import org.apache.flink.table.test.program.SinkTestStep;
+import org.apache.flink.table.test.program.SourceTestStep;
+import org.apache.flink.table.test.program.TableTestProgram;
+import org.apache.flink.table.utils.DateTimeUtils;
+import org.apache.flink.types.Row;
+
+/** {@link TableTestProgram} definitions for testing {@link StreamExecTableSourceScan}. */
+public class TableSourceScanTestPrograms {
+
+    static final Row[] BEFORE_DATA = {
+        Row.of(1, 1L, "hi", DateTimeUtils.toLocalDateTime(1586937601000L)),
+        Row.of(2, 2L, "hello", DateTimeUtils.toLocalDateTime(1586937602000L)),
+        Row.of(3, 2L, "hello world", DateTimeUtils.toLocalDateTime(1586937603000L))
+    };
+
+    static final Row[] AFTER_DATA = {
+        Row.of(4, 4L, "foo", DateTimeUtils.toLocalDateTime(1586937614000L)),
+        Row.of(5, 2L, "foo bar", DateTimeUtils.toLocalDateTime(1586937615000L)),
+    };
+
+    static final TableTestProgram PROJECT_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-project-pushdown",
+                            "validates table source scan with project pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")
+                    .build();
+
+    static final TableTestProgram PROJECT_PUSHDOWN_DISABLED =
+            TableTestProgram.of(
+                            "table-source-scan-project-push-down-disabled",
+                            "validates table source scan with project pushdown disabled")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema(
+                                            "a INT",
+                                            "b BIGINT",
+                                            "c VARCHAR",
+                                            "ts TIMESTAMP(3) METADATA")
+                                    .addOption("readable-metadata", "ts:TIMESTAMP(3)")
+                                    .addOption("enable-projection-push-down", "false")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")
+                    .build();
+
+    static final TableTestProgram FILTER_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-filter-pushdown",
+                            "validates table source scan with filter pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .addOption("filterable-fields", "a")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[2, 2, hello]", "+I[3, 2, hello world]")
+                                    .consumedAfterRestore("+I[4, 4, foo]", "+I[5, 2, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT * FROM source_t WHERE a > 1")
+                    .build();
+
+    static final TableTestProgram LIMIT_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-limit-pushdown",
+                            "validates table source scan with limit pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore("+I[1, 1, hi]", "+I[2, 2, hello]")
+                                    .consumedAfterRestore(new String[] {})
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, b, c FROM source_t LIMIT 2")
+                    .build();
+
+    static final TableTestProgram PARTITION_PUSHDOWN =
+            TableTestProgram.of(
+                            "table-source-scan-partition-pushdown",
+                            "validates table source scan with partition pushdown")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .addPartitionKeys("b")
+                                    .addOption("partition-list", "b:1,b:2")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "b BIGINT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[2, 2, hello]", "+I[3, 2, hello world]")
+                                    .consumedAfterRestore("+I[5, 2, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, b, c FROM source_t WHERE b = 2")
+                    .build();
+
+    static final TableTestProgram READING_METADATA =
+            TableTestProgram.of(
+                            "table-source-scan-reading-metadata",
+                            "validates table source scan by reading metadata")
+                    .setupTableSource(
+                            SourceTestStep.newBuilder("source_t")
+                                    .addSchema(
+                                            "a INT",
+                                            "b BIGINT",
+                                            "c VARCHAR",
+                                            "d TIMESTAMP(3) METADATA")
+                                    .addOption("readable-metadata", "d:TIMESTAMP(3)")
+                                    .producedBeforeRestore(BEFORE_DATA)
+                                    .producedAfterRestore(AFTER_DATA)
+                                    .build())
+                    .setupTableSink(
+                            SinkTestStep.newBuilder("sink_t")
+                                    .addSchema("a INT", "c VARCHAR")
+                                    .consumedBeforeRestore(
+                                            "+I[1, hi]", "+I[2, hello]", "+I[3, hello world]")
+                                    .consumedAfterRestore("+I[4, foo]", "+I[5, foo bar]")
+                                    .build())
+                    .runSql("INSERT INTO sink_t SELECT a, c FROM source_t")

Review Comment:
   Thanks. Fixed it by selecting the metadata column in the query



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "jnh5y (via GitHub)" <gi...@apache.org>.
jnh5y commented on code in PR #24020:
URL: https://github.com/apache/flink/pull/24020#discussion_r1442077875


##########
flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java:
##########
@@ -198,6 +198,18 @@ public SqlTestStep getRunSqlTestStep() {
         return (SqlTestStep) sqlSteps.get(0);
     }
 
+    /**
+     * Convenience method to avoid boilerplate code. It assumes only one statement set is tested.
+     */
+    public StatementSetTestStep getRunStatementSetTestStep() {
+        List<TestStep> statementSetSteps =
+                runSteps.stream()
+                        .filter(s -> s.getKind() == TestKind.STATEMENT_SET)
+                        .collect(Collectors.toList());
+
+        return (StatementSetTestStep) statementSetSteps.get(0);

Review Comment:
   Any particular reason to assume that there is only one StatementSet being tested?



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "bvarghese1 (via GitHub)" <gi...@apache.org>.
bvarghese1 commented on code in PR #24020:
URL: https://github.com/apache/flink/pull/24020#discussion_r1442091512


##########
flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java:
##########
@@ -198,6 +198,18 @@ public SqlTestStep getRunSqlTestStep() {
         return (SqlTestStep) sqlSteps.get(0);
     }
 
+    /**
+     * Convenience method to avoid boilerplate code. It assumes only one statement set is tested.
+     */
+    public StatementSetTestStep getRunStatementSetTestStep() {
+        List<TestStep> statementSetSteps =
+                runSteps.stream()
+                        .filter(s -> s.getKind() == TestKind.STATEMENT_SET)
+                        .collect(Collectors.toList());
+
+        return (StatementSetTestStep) statementSetSteps.get(0);

Review Comment:
   For tests, as a convenience we can assume there will be only 1 statement set which can contain multiple sql statements.



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "dawidwys (via GitHub)" <gi...@apache.org>.
dawidwys merged PR #24020:
URL: https://github.com/apache/flink/pull/24020


-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] [FLINK-33969] Implement restore tests for TableSourceScan node [flink]

Posted by "bvarghese1 (via GitHub)" <gi...@apache.org>.
bvarghese1 commented on code in PR #24020:
URL: https://github.com/apache/flink/pull/24020#discussion_r1447721761


##########
flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java:
##########
@@ -198,6 +198,18 @@ public SqlTestStep getRunSqlTestStep() {
         return (SqlTestStep) sqlSteps.get(0);
     }
 
+    /**
+     * Convenience method to avoid boilerplate code. It assumes only one statement set is tested.
+     */
+    public StatementSetTestStep getRunStatementSetTestStep() {
+        List<TestStep> statementSetSteps =
+                runSteps.stream()
+                        .filter(s -> s.getKind() == TestKind.STATEMENT_SET)
+                        .collect(Collectors.toList());
+
+        return (StatementSetTestStep) statementSetSteps.get(0);

Review Comment:
   Added a precondition check to verify similar to `getRunSqlTestStep`



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org