dawidwys commented on code in PR #23680:
URL: https://github.com/apache/flink/pull/23680#discussion_r1400462743


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java:
##########
@@ -151,7 +156,7 @@ public void generateTestSetupFiles(TableTestProgram 
program) throws Exception {
                     (integer, strings) -> {
                         final boolean shouldTakeSavepoint =
                                 CollectionUtils.isEqualCollection(
-                                        
TestValuesTableFactory.getRawResultsAsStrings(tableName),
+                                        
TestValuesTableFactory.getResultsAsStrings(tableName),

Review Comment:
   I think this should be configurable. We should still have the choice to 
assert on raw results.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java:
##########
@@ -280,6 +280,15 @@ public static void registerLocalRawResultsObserver(
         TestValuesRuntimeFunctions.registerLocalRawResultsObserver(tableName, 
observer);
     }
 
+    /**
+     * Removes observers for a table.
+     *
+     * @param tableName the table name of the registered table sink.
+     */
+    public static void unregisterLocalRawResultsObserver(String tableName) {

Review Comment:
   nit:
   ```suggestion
       public static void clearLocalRawResultsObserver(String tableName) {
   ```
   
   Unregister would suggest we can unregister a particular instance of an 
observer. At least that would be my assumption.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java:
##########
@@ -78,6 +80,7 @@
  */
 @ExtendWith(MiniClusterExtension.class)
 @TestInstance(TestInstance.Lifecycle.PER_CLASS)
+@TestMethodOrder(org.junit.jupiter.api.MethodOrderer.OrderAnnotation.class)

Review Comment:
   nit: import `OrderAnnotation`



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/JoinTestPrograms.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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.testutils;
+
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin;
+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.types.Row;
+
+/** {@link TableTestProgram} definitions for testing {@link StreamExecJoin}. */
+public class JoinTestPrograms {
+    static final TableTestProgram NON_WINDOW_INNER_JOIN;
+    static final TableTestProgram NON_WINDOW_INNER_JOIN_WITH_NULL;
+    static final TableTestProgram CROSS_JOIN;
+    static final TableTestProgram JOIN_WITH_FILTER;
+    static final TableTestProgram INNER_JOIN_WITH_DUPLICATE_KEY;
+    static final TableTestProgram INNER_JOIN_WITH_NON_EQUI_JOIN;
+    static final TableTestProgram INNER_JOIN_WITH_EQUAL_PK;
+    static final TableTestProgram INNER_JOIN_WITH_PK;
+    static final TableTestProgram LEFT_JOIN;
+    static final TableTestProgram FULL_OUTER;
+    static final TableTestProgram RIGHT_JOIN;
+    static final TableTestProgram SEMI_JOIN;
+    static final TableTestProgram ANTI_JOIN;
+
+    static final SourceTestStep EMPLOYEE =
+            SourceTestStep.newBuilder("EMPLOYEE")
+                    .addSchema("deptno int", "salary bigint", "name varchar")
+                    .addOption("filterable-fields", "salary")
+                    .producedBeforeRestore(
+                            Row.of(null, 101L, "Adam"),
+                            Row.of(1, 1L, "Baker"),
+                            Row.of(2, 2L, "Charlie"),
+                            Row.of(3, 2L, "Don"),
+                            Row.of(7, 6L, "Victor"))
+                    .producedAfterRestore(Row.of(4, 3L, "Juliet"))
+                    .build();
+
+    static final SourceTestStep DEPARTMENT =
+            SourceTestStep.newBuilder("DEPARTMENT")
+                    .addSchema(
+                            "department_num int", "b2 bigint", "b3 int", 
"department_name varchar")
+                    .producedBeforeRestore(
+                            Row.of(null, 102L, 0, "Accounting"),
+                            Row.of(1, 1L, 0, "Research"),
+                            Row.of(2, 2L, 1, "Human Resources"),
+                            Row.of(2, 3L, 2, "HR"),
+                            Row.of(3, 1L, 2, "Sales"))
+                    .producedAfterRestore(Row.of(2, 4L, 3, "People 
Operations"))
+                    .build();
+
+    static final SourceTestStep DEPARTMENT_NONULLS =
+            SourceTestStep.newBuilder("DEPARTMENT")
+                    .addSchema(
+                            "department_num int", "b2 bigint", "b3 int", 
"department_name varchar")
+                    .producedBeforeRestore(
+                            Row.of(1, 1L, 0, "Research"),
+                            Row.of(2, 2L, 1, "Human Resources"),
+                            Row.of(2, 3L, 2, "HR"),
+                            Row.of(3, 1L, 2, "Sales"))
+                    .producedAfterRestore(Row.of(2, 4L, 3, "People 
Operations"))
+                    .build();
+    static final SourceTestStep SOURCE_T1 =
+            SourceTestStep.newBuilder("T1")
+                    .addSchema("a int", "b bigint", "c varchar")
+                    .producedBeforeRestore(
+                            Row.of(1, 1L, "Baker1"),
+                            Row.of(1, 2L, "Baker2"),
+                            Row.of(1, 2L, "Baker2"),
+                            Row.of(1, 5L, "Baker3"),
+                            Row.of(2, 7L, "Baker5"),
+                            Row.of(1, 9L, "Baker6"),
+                            Row.of(1, 8L, "Baker8"),
+                            Row.of(3, 8L, "Baker9"))
+                    .producedAfterRestore(Row.of(1, 1L, "PostRestore"))
+                    .build();
+    static final SourceTestStep SOURCE_T2 =
+            SourceTestStep.newBuilder("T2")
+                    .addSchema("a int", "b bigint", "c varchar")
+                    .producedBeforeRestore(
+                            Row.of(1, 1L, "BakerBaker"),
+                            Row.of(2, 2L, "HeHe"),
+                            Row.of(3, 2L, "HeHe"))
+                    .producedAfterRestore(Row.of(2, 1L, "PostRestoreRight"))
+                    .build();
+
+    static {
+        NON_WINDOW_INNER_JOIN =
+                TableTestProgram.of("non-window-inner-join", "test non-window 
inner join")
+                        .setupTableSource(SOURCE_T1)
+                        .setupTableSource(SOURCE_T2)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("a int", "c1 varchar", "c2 
varchar")
+                                        .consumedBeforeRestore(
+                                                Row.of(1, "BakerBaker", 
"Baker2"),
+                                                Row.of(1, "BakerBaker", 
"Baker2"),
+                                                Row.of(1, "BakerBaker", 
"Baker3"),
+                                                Row.of(2, "HeHe", "Baker5"),
+                                                Row.of(1, "BakerBaker", 
"Baker6"),
+                                                Row.of(1, "BakerBaker", 
"Baker8"))
+                                        .consumedAfterRestore(
+                                                Row.of(2, "PostRestoreRight", 
"Baker5"))
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT t2.a, t2.c, t1.c\n"
+                                        + "FROM (\n"
+                                        + " SELECT if(a = 3, cast(null as 
int), a) as a, b, c FROM T1\n"
+                                        + ") as t1\n"
+                                        + "JOIN (\n"
+                                        + " SELECT if(a = 3, cast(null as 
int), a) as a, b, c FROM T2\n"
+                                        + ") as t2\n"
+                                        + "ON t1.a = t2.a AND t1.b > t2.b")
+                        .build();
+
+        NON_WINDOW_INNER_JOIN_WITH_NULL =
+                TableTestProgram.of(
+                                "non-window-inner-join-with-null-cond",
+                                "test non-window inner join")
+                        .setupTableSource(SOURCE_T1)
+                        .setupTableSource(SOURCE_T2)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("a int", "c1 varchar", "c2 
varchar")
+                                        .consumedBeforeRestore(
+                                                Row.of(1, "BakerBaker", 
"Baker2"),
+                                                Row.of(1, "BakerBaker", 
"Baker2"),
+                                                Row.of(1, "BakerBaker", 
"Baker3"),
+                                                Row.of(2, "HeHe", "Baker5"),
+                                                Row.of(1, "BakerBaker", 
"Baker6"),
+                                                Row.of(1, "BakerBaker", 
"Baker8"),
+                                                Row.of(null, "HeHe", "Baker9"))
+                                        .consumedAfterRestore(
+                                                Row.of(2, "PostRestoreRight", 
"Baker5"))
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT t2.a, t2.c, t1.c\n"
+                                        + "FROM (\n"
+                                        + " SELECT if(a = 3, cast(null as 
int), a) as a, b, c FROM T1\n"
+                                        + ") as t1\n"
+                                        + "JOIN (\n"
+                                        + " SELECT if(a = 3, cast(null as 
int), a) as a, b, c FROM T2\n"
+                                        + ") as t2\n"
+                                        + "ON \n"
+                                        + "  ((t1.a is null AND t2.a is null) 
OR\n"
+                                        + "  (t1.a = t2.a))\n"
+                                        + "  AND t1.b > t2.b")
+                        .build();
+
+        CROSS_JOIN =
+                TableTestProgram.of("cross-join", "test cross join")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("name varchar", 
"department_name varchar")
+                                        .consumedBeforeRestore(
+                                                Row.of("Adam", "Accounting"),
+                                                Row.of("Baker", "Accounting"),
+                                                Row.of("Adam", "Research"),
+                                                Row.of("Baker", "Research"),
+                                                Row.of("Charlie", 
"Accounting"),
+                                                Row.of("Charlie", "Research"),
+                                                Row.of("Charlie", "Human 
Resources"),
+                                                Row.of("Adam", "Human 
Resources"),
+                                                Row.of("Baker", "Human 
Resources"),
+                                                Row.of("Don", "Accounting"),
+                                                Row.of("Don", "Human 
Resources"),
+                                                Row.of("Don", "Research"),
+                                                Row.of("Victor", "Accounting"),
+                                                Row.of("Victor", "Human 
Resources"),
+                                                Row.of("Victor", "Research"),
+                                                Row.of("Don", "HR"),
+                                                Row.of("Charlie", "HR"),
+                                                Row.of("Adam", "HR"),
+                                                Row.of("Baker", "HR"),
+                                                Row.of("Victor", "HR"),
+                                                Row.of("Don", "Sales"),
+                                                Row.of("Charlie", "Sales"),
+                                                Row.of("Adam", "Sales"),
+                                                Row.of("Baker", "Sales"),
+                                                Row.of("Victor", "Sales"))
+                                        .consumedAfterRestore(
+                                                Row.of("Juliet", "Human 
Resources"),
+                                                Row.of("Juliet", "Sales"),
+                                                Row.of("Juliet", "Research"),
+                                                Row.of("Juliet", "Accounting"),
+                                                Row.of("Juliet", "HR"),
+                                                Row.of("Juliet", "People 
Operations"),
+                                                Row.of("Victor", "People 
Operations"),
+                                                Row.of("Charlie", "People 
Operations"),
+                                                Row.of("Baker", "People 
Operations"),
+                                                Row.of("Adam", "People 
Operations"),
+                                                Row.of("Don", "People 
Operations"))
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT name, department_name FROM 
EMPLOYEE, DEPARTMENT")
+                        .build();
+
+        JOIN_WITH_FILTER =
+                TableTestProgram.of("join-with-filter", "test join with 
filter")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("name varchar", 
"department_name varchar")
+                                        .consumedBeforeRestore(
+                                                Row.of("Baker", "Research"),
+                                                Row.of("Baker", "Sales"))
+                                        .consumedAfterRestore(new Row[] {})
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT name, department_name FROM 
EMPLOYEE, DEPARTMENT where salary = b2 and salary < CAST(2 AS BIGINT)")
+                        .build();
+
+        INNER_JOIN_WITH_DUPLICATE_KEY =
+                TableTestProgram.of(
+                                "inner-join-with-duplicate-key", "inner join 
with duplicate key")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("deptno int", 
"department_num int")
+                                        .consumedBeforeRestore(Row.of(2, 2))
+                                        .consumedAfterRestore(new Row[] {})
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT deptno, department_num FROM 
EMPLOYEE JOIN DEPARTMENT ON deptno = department_num AND deptno = b3")
+                        .build();
+
+        INNER_JOIN_WITH_NON_EQUI_JOIN =
+                TableTestProgram.of(
+                                "inner-join-with-non-equi-join", "inner join 
with non-equi join")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("name varchar", 
"department_name varchar")
+                                        .consumedBeforeRestore(Row.of("Don", 
"Sales"))
+                                        .consumedAfterRestore(new Row[] {})
+                                        .build())
+                        .runSql(
+                                "insert into MySink "
+                                        + "SELECT name, department_name FROM 
EMPLOYEE JOIN DEPARTMENT ON deptno = department_num AND salary > b2")
+                        .build();
+
+        String query1 = "SELECT SUM(salary) AS salary, deptno FROM EMPLOYEE 
GROUP BY deptno";
+        String query2 =
+                "SELECT SUM(b2) AS b2, department_num FROM DEPARTMENT GROUP BY 
department_num";
+
+        INNER_JOIN_WITH_EQUAL_PK =
+                TableTestProgram.of("inner-join-with-equal-pk", "inner join 
with equal pk")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("deptno int", 
"department_num int")
+                                        .consumedBeforeRestore(
+                                                Row.of(1, 1), Row.of(2, 2), 
Row.of(3, 3))
+                                        .consumedAfterRestore(new Row[] {})
+                                        .build())
+                        .runSql(
+                                String.format(
+                                        "INSERT INTO MySink SELECT deptno, 
department_num FROM (%s) JOIN (%s) ON deptno = department_num",
+                                        query1, query2))
+                        .build();
+
+        INNER_JOIN_WITH_PK =
+                TableTestProgram.of("inner-join-with-pk", "inner join with pk")
+                        .setupTableSource(EMPLOYEE)
+                        .setupTableSource(DEPARTMENT)
+                        .setupTableSink(
+                                SinkTestStep.newBuilder("MySink")
+                                        .addSchema("deptno int", 
"department_num int")
+                                        .consumedBeforeRestore(
+                                                Row.of(1, 1),
+                                                //
+                                                // Row.of(7, 2),

Review Comment:
   What is this comment for?



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java:
##########
@@ -181,12 +186,25 @@ public void generateTestSetupFiles(TableTestProgram 
program) throws Exception {
         CommonTestUtils.waitForJobStatus(jobClient, 
Collections.singletonList(JobStatus.FINISHED));
         final Path savepointPath = Paths.get(new URI(savepoint));
         final Path savepointDirPath = getSavepointPath(program, 
getLatestMetadata());
-        Files.createDirectories(savepointDirPath);
+        // Delete directory savepointDirPath if it already exists
+        if (Files.exists(savepointDirPath)) {
+            
Files.walk(savepointDirPath).map(Path::toFile).forEach(java.io.File::delete);
+        } else {
+            Files.createDirectories(savepointDirPath);
+        }

Review Comment:
   I am a bit hesitant to making it easier to regenerate the test files. If the 
process requires manual action it is more likely people won't simply generate 
the files again on incompatible changes.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java:
##########
@@ -224,7 +242,9 @@ void testRestore(TableTestProgram program, ExecNodeMetadata 
metadata) throws Exc
                 tEnv.loadPlan(PlanReference.fromFile(getPlanPath(program, 
metadata)));
         compiledPlan.execute().await();
         for (SinkTestStep sinkTestStep : program.getSetupSinkTestSteps()) {
-            
assertThat(TestValuesTableFactory.getRawResultsAsStrings(sinkTestStep.name))
+            // TODO: Fix this.  If the records after a restore cause 
retractions,
+            // this approach will not work.
+            
assertThat(TestValuesTableFactory.getResultsAsStrings(sinkTestStep.name))

Review Comment:
   I think this should be configurable. We should still have the choice to 
assert on raw results.



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/JoinTestPrograms.java:
##########
@@ -0,0 +1,409 @@
+/*
+ * 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.testutils;
+
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin;
+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.types.Row;
+
+/** {@link TableTestProgram} definitions for testing {@link StreamExecJoin}. */
+public class JoinTestPrograms {

Review Comment:
   Is this expected to replace `JoinJsonPlanTest`? I see you did not remove the 
file.



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to