dawidwys commented on code in PR #23814: URL: https://github.com/apache/flink/pull/23814#discussion_r1409505697
########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinTestPrograms.java: ########## @@ -0,0 +1,287 @@ +/* + * 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.types.Row; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecGroupWindowAggregate}. */ Review Comment: Please update the javadoc. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinTestPrograms.java: ########## @@ -0,0 +1,287 @@ +/* + * 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.types.Row; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecGroupWindowAggregate}. */ +public class LookupJoinTestPrograms { + + static final SourceTestStep CUSTOMERS = + SourceTestStep.newBuilder("customers_t") // static table + .addOption("disable-lookup", "false") + .addOption("filterable-fields", "age") + .addSchema( + "id INT PRIMARY KEY NOT ENFORCED", + "name STRING", + "age INT", + "city STRING", + "state STRING", + "zipcode INT") + .producedBeforeRestore( + Row.of(1, "Bob", 28, "Mountain View", "California", 94043), + Row.of(2, "Alice", 32, "San Francisco", "California", 95016), + Row.of(3, "Claire", 37, "Austin", "Texas", 73301), + Row.of(4, "Shannon", 29, "Boise", "Idaho", 83701), + Row.of(5, "Jake", 42, "New York City", "New York", 10001)) + // Note: Before data state is not persisted for static tables during savepoint + .producedAfterRestore( + Row.of(1, "Bob", 28, "San Jose", "California", 94089), + Row.of(6, "Joana", 54, "Atlanta", "Georgia", 30033)) Review Comment: Does it make sense to have different data before and after the restore? I am fine if you think it is. I'd rephrase the `Note` though, as I think in the current shape it's a bit confusing. Lookup tables do not act as regular tables and they do not consume any of the data. Therefore there is nothing to "persist". Lookup tables work in a way that a value is looked up in the external system each time a record is received on the other side of the join. `produced***` might be confusing in that context, but adding new methods for that single use case might be an overkill. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java: ########## @@ -1,273 +0,0 @@ -/* - * 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.api.common.typeinfo.Types; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.table.planner.runtime.utils.InMemoryLookupableTableSource; -import org.apache.flink.table.planner.utils.StreamTableTestUtil; -import org.apache.flink.table.planner.utils.TableTestBase; -import org.apache.flink.types.Row; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; - -import scala.collection.JavaConverters; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Test json serialization/deserialization for LookupJoin. */ -class LookupJoinJsonPlanTest extends TableTestBase { - - private StreamTableTestUtil util; - private TableEnvironment tEnv; - - @BeforeEach - void setup() { - util = streamTestUtil(TableConfig.getDefault()); - tEnv = util.getTableEnv(); - - String srcTableA = - "CREATE TABLE MyTable (\n" - + " a int,\n" - + " b varchar,\n" - + " c bigint,\n" - + " proctime as PROCTIME(),\n" - + " rowtime as TO_TIMESTAMP(FROM_UNIXTIME(c)),\n" - + " watermark for rowtime as rowtime - INTERVAL '1' second \n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - String srcTableB = - "CREATE TABLE LookupTable (\n" - + " id int,\n" - + " name varchar,\n" - + " age int \n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - String sinkTable1 = - "CREATE TABLE Sink1 (\n" - + " a int,\n" - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false')"; - String sinkTable2 = - "CREATE TABLE MySink1 (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(srcTableA); - tEnv.executeSql(srcTableB); - tEnv.executeSql(sinkTable1); - tEnv.executeSql(sinkTable2); - } - - @Test - void testJoinTemporalTable() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "INSERT INTO MySink SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithProjectionPushDown() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "INSERT INTO MySink \n" - + "SELECT T.*, D.id \n" - + "FROM MyTable AS T \n" - + "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D \n" - + "ON T.a = D.id\n"); - } - - @Test - void testLegacyTableSourceException() { - TableSchema tableSchema = - TableSchema.builder() - .field("id", Types.INT) - .field("name", Types.STRING) - .field("age", Types.INT) - .build(); - InMemoryLookupableTableSource.createTemporaryTable( - tEnv, - false, - JavaConverters.asScalaIteratorConverter(new ArrayList<Row>().iterator()) - .asScala() - .toList(), - tableSchema, - "LookupTable", - true); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - assertThatThrownBy( - () -> - util.verifyJsonPlan( - "INSERT INTO MySink SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id")) - .satisfies( - anyCauseMatches( - ValidationException.class, - "TemporalTableSourceSpec can not be serialized.")); - } - - @Test - void testAggAndLeftJoinWithTryResolveMode() { - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, - OptimizerConfigOptions.NonDeterministicUpdateStrategy.TRY_RESOLVE); - - util.verifyJsonPlan( - "INSERT INTO Sink1 " - + "SELECT T.a, D.name, D.age " - + "FROM (SELECT max(a) a, count(c) c, PROCTIME() proctime FROM MyTable GROUP BY b) T " - + "LEFT JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncHint() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'output-mode'='allow_unordered') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncHint2() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'timeout'='600s', 'capacity'='1000') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithRetryHint() { - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncRetryHint() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncRetryHint2() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'timeout'='600s', 'capacity'='1000', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testLeftJoinTemporalTableWithPreFilter() { - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT * " - + "FROM MyTable AS T LEFT JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b > 'abc'"); - } - - @Test - void testLeftJoinTemporalTableWithPostFilter() { - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT * " - + "FROM MyTable AS T LEFT JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id " - + "AND CHAR_LENGTH(D.name) > CHAR_LENGTH(T.b)"); - } - - @Test - void testLeftJoinTemporalTableWithMultiJoinConditions() { Review Comment: Is this covered? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinTestPrograms.java: ########## @@ -0,0 +1,287 @@ +/* + * 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.types.Row; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecGroupWindowAggregate}. */ +public class LookupJoinTestPrograms { + + static final SourceTestStep CUSTOMERS = + SourceTestStep.newBuilder("customers_t") // static table + .addOption("disable-lookup", "false") + .addOption("filterable-fields", "age") + .addSchema( + "id INT PRIMARY KEY NOT ENFORCED", + "name STRING", + "age INT", + "city STRING", + "state STRING", + "zipcode INT") + .producedBeforeRestore( + Row.of(1, "Bob", 28, "Mountain View", "California", 94043), + Row.of(2, "Alice", 32, "San Francisco", "California", 95016), + Row.of(3, "Claire", 37, "Austin", "Texas", 73301), + Row.of(4, "Shannon", 29, "Boise", "Idaho", 83701), + Row.of(5, "Jake", 42, "New York City", "New York", 10001)) + // Note: Before data state is not persisted for static tables during savepoint + .producedAfterRestore( + Row.of(1, "Bob", 28, "San Jose", "California", 94089), + Row.of(6, "Joana", 54, "Atlanta", "Georgia", 30033)) + .build(); + + static final SourceTestStep ORDERS = + SourceTestStep.newBuilder("orders_t") + .addOption("filterable-fields", "customer_id") + .addSchema( + "order_id INT", + "customer_id INT", + "total DOUBLE", + "order_time STRING", + "proc_time AS PROCTIME()") + .producedBeforeRestore( + Row.of(1, 3, 44.44, "2020-10-10 00:00:01"), + Row.of(2, 5, 100.02, "2020-10-10 00:00:02"), + Row.of(4, 2, 92.61, "2020-10-10 00:00:04"), + Row.of(3, 1, 23.89, "2020-10-10 00:00:03"), + Row.of(6, 4, 7.65, "2020-10-10 00:00:06"), + Row.of(5, 2, 12.78, "2020-10-10 00:00:05")) + .producedAfterRestore( + Row.of(7, 6, 17.58, "2020-10-10 00:00:07"), // new customer + Row.of(9, 1, 143.21, "2020-10-10 00:00:08") // updated zip code + ) + .build(); + + static final List<String> SINK_SCHEMA = + Arrays.asList( + "order_id INT", + "total DOUBLE", + "id INT", + "name STRING", + "age INT", + "city STRING", + "state STRING", + "zipcode INT"); + + static final TableTestProgram LOOKUP_JOIN_PROJECT_PUSHDOWN = + TableTestProgram.of( + "lookup-join-project-pushdown", + "validates lookup join with project pushdown") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema( + SINK_SCHEMA.stream() + .filter(field -> !field.equals("age INT")) + .collect(Collectors.toList())) + .consumedBeforeRestore( + "+I[1, 44.44, 3, Claire, Austin, Texas, 73301]", + "+I[2, 100.02, 5, Jake, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, San Francisco, California, 95016]", + "+I[3, 23.89, 1, Bob, Mountain View, California, 94043]", + "+I[6, 7.65, 4, Shannon, Boise, Idaho, 83701]", + "+I[5, 12.78, 2, Alice, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, Atlanta, Georgia, 30033]", + "+I[9, 143.21, 1, Bob, San Jose, California, 94089]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id") + .build(); + + static final TableTestProgram LOOKUP_JOIN_FILTER_PUSHDOWN = + TableTestProgram.of( + "lookup-join-filter-pushdown", + "validates lookup join with filter pushdown") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[1, 44.44, 3, Claire, 37, Austin, Texas, 73301]", + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.age, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id AND C.age > 30") + .build(); + + static final TableTestProgram LOOKUP_JOIN_PRE_FILTER = + TableTestProgram.of("lookup-join-pre-filter", "validates lookup join with pre filter") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.age, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id AND C.age > 30 AND O.customer_id <> 3") + .build(); + + static final TableTestProgram LOOKUP_JOIN_LEFT_JOIN = + TableTestProgram.of("lookup-join-left-join", "validates lookup join with left join") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[1, 44.44, null, null, null, null, null, null]", + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[3, 23.89, null, null, null, null, null, null]", + "+I[6, 7.65, null, null, null, null, null, null]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]", + "+I[9, 143.21, null, null, null, null, null, null]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.age, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "LEFT JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id AND C.age > 30 AND O.customer_id <> 3") + .build(); + + static final TableTestProgram LOOKUP_JOIN_ASYNC_HINT = + TableTestProgram.of("lookup-join-async-hint", "validates lookup join with async hint") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[1, 44.44, 3, Claire, 37, Austin, Texas, 73301]", + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[3, 23.89, 1, Bob, 28, Mountain View, California, 94043]", + "+I[6, 7.65, 4, Shannon, 29, Boise, Idaho, 83701]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]", + "+I[9, 143.21, 1, Bob, 28, San Jose, California, 94089]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "/*+ LOOKUP('table'='C', 'async'='true', 'output-mode'='allow_unordered') */ " Review Comment: Do you know if those options are stored in the CompiledPlan? I looked shortly into the `StreamExecLookupJoin` and I believe e.g. `async` should be serialized in `asyncOptions`, but I can't find such a field in any of the plans (even the old ones). ########## flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java: ########## @@ -56,7 +56,7 @@ public TableResult apply(TableEnvironment env) { public TableResult apply(TableEnvironment env, Map<String, String> extraOptions) { final Map<String, String> allOptions = new HashMap<>(options); - allOptions.putAll(extraOptions); Review Comment: `extraOptions` should take precedence. The idea was that test base can enforce certain properties. Why do we need to change that? ########## flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java: ########## @@ -56,7 +56,7 @@ public TableResult apply(TableEnvironment env) { public TableResult apply(TableEnvironment env, Map<String, String> extraOptions) { final Map<String, String> allOptions = new HashMap<>(options); - allOptions.putAll(extraOptions); Review Comment: I believe that's to overwrite`disable-lookup`? I think we can remove it from `RestoreTestBase`. It might've been unnecessary there. ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest.java: ########## @@ -1,273 +0,0 @@ -/* - * 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.api.common.typeinfo.Types; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.config.OptimizerConfigOptions; -import org.apache.flink.table.planner.runtime.utils.InMemoryLookupableTableSource; -import org.apache.flink.table.planner.utils.StreamTableTestUtil; -import org.apache.flink.table.planner.utils.TableTestBase; -import org.apache.flink.types.Row; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; - -import scala.collection.JavaConverters; - -import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -/** Test json serialization/deserialization for LookupJoin. */ -class LookupJoinJsonPlanTest extends TableTestBase { - - private StreamTableTestUtil util; - private TableEnvironment tEnv; - - @BeforeEach - void setup() { - util = streamTestUtil(TableConfig.getDefault()); - tEnv = util.getTableEnv(); - - String srcTableA = - "CREATE TABLE MyTable (\n" - + " a int,\n" - + " b varchar,\n" - + " c bigint,\n" - + " proctime as PROCTIME(),\n" - + " rowtime as TO_TIMESTAMP(FROM_UNIXTIME(c)),\n" - + " watermark for rowtime as rowtime - INTERVAL '1' second \n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - String srcTableB = - "CREATE TABLE LookupTable (\n" - + " id int,\n" - + " name varchar,\n" - + " age int \n" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'bounded' = 'false')"; - String sinkTable1 = - "CREATE TABLE Sink1 (\n" - + " a int,\n" - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'sink-insert-only' = 'false')"; - String sinkTable2 = - "CREATE TABLE MySink1 (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(srcTableA); - tEnv.executeSql(srcTableB); - tEnv.executeSql(sinkTable1); - tEnv.executeSql(sinkTable2); - } - - @Test - void testJoinTemporalTable() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "INSERT INTO MySink SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithProjectionPushDown() { - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - util.verifyJsonPlan( - "INSERT INTO MySink \n" - + "SELECT T.*, D.id \n" - + "FROM MyTable AS T \n" - + "JOIN LookupTable FOR SYSTEM_TIME AS OF T.proctime AS D \n" - + "ON T.a = D.id\n"); - } - - @Test - void testLegacyTableSourceException() { - TableSchema tableSchema = - TableSchema.builder() - .field("id", Types.INT) - .field("name", Types.STRING) - .field("age", Types.INT) - .build(); - InMemoryLookupableTableSource.createTemporaryTable( - tEnv, - false, - JavaConverters.asScalaIteratorConverter(new ArrayList<Row>().iterator()) - .asScala() - .toList(), - tableSchema, - "LookupTable", - true); - String sinkTableDdl = - "CREATE TABLE MySink (\n" - + " a int,\n" - + " b varchar," - + " c bigint," - + " proctime timestamp(3)," - + " rowtime timestamp(3)," - + " id int," - + " name varchar," - + " age int" - + ") with (\n" - + " 'connector' = 'values',\n" - + " 'table-sink-class' = 'DEFAULT')"; - tEnv.executeSql(sinkTableDdl); - assertThatThrownBy( - () -> - util.verifyJsonPlan( - "INSERT INTO MySink SELECT * FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id")) - .satisfies( - anyCauseMatches( - ValidationException.class, - "TemporalTableSourceSpec can not be serialized.")); - } - - @Test - void testAggAndLeftJoinWithTryResolveMode() { - tEnv.getConfig() - .set( - OptimizerConfigOptions.TABLE_OPTIMIZER_NONDETERMINISTIC_UPDATE_STRATEGY, - OptimizerConfigOptions.NonDeterministicUpdateStrategy.TRY_RESOLVE); - - util.verifyJsonPlan( - "INSERT INTO Sink1 " - + "SELECT T.a, D.name, D.age " - + "FROM (SELECT max(a) a, count(c) c, PROCTIME() proctime FROM MyTable GROUP BY b) T " - + "LEFT JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncHint() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'output-mode'='allow_unordered') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncHint2() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'timeout'='600s', 'capacity'='1000') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithRetryHint() { - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncRetryHint() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testJoinTemporalTableWithAsyncRetryHint2() { - // LookupTable has sync func only, just verify the hint has take effect - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT " - + "/*+ LOOKUP('table'='D', 'async'='true', 'timeout'='600s', 'capacity'='1000', 'retry-predicate'='lookup_miss', 'retry-strategy'='fixed_delay', 'fixed-delay'='10s', 'max-attempts'='3') */ * " - + "FROM MyTable AS T JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id"); - } - - @Test - void testLeftJoinTemporalTableWithPreFilter() { - util.verifyJsonPlan( - "INSERT INTO MySink1 SELECT * " - + "FROM MyTable AS T LEFT JOIN LookupTable " - + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.a = D.id AND T.b > 'abc'"); - } - - @Test - void testLeftJoinTemporalTableWithPostFilter() { Review Comment: Is this covered? ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinTestPrograms.java: ########## @@ -0,0 +1,287 @@ +/* + * 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.types.Row; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecGroupWindowAggregate}. */ +public class LookupJoinTestPrograms { + + static final SourceTestStep CUSTOMERS = + SourceTestStep.newBuilder("customers_t") // static table + .addOption("disable-lookup", "false") + .addOption("filterable-fields", "age") + .addSchema( + "id INT PRIMARY KEY NOT ENFORCED", + "name STRING", + "age INT", + "city STRING", + "state STRING", + "zipcode INT") + .producedBeforeRestore( + Row.of(1, "Bob", 28, "Mountain View", "California", 94043), + Row.of(2, "Alice", 32, "San Francisco", "California", 95016), + Row.of(3, "Claire", 37, "Austin", "Texas", 73301), + Row.of(4, "Shannon", 29, "Boise", "Idaho", 83701), + Row.of(5, "Jake", 42, "New York City", "New York", 10001)) + // Note: Before data state is not persisted for static tables during savepoint + .producedAfterRestore( + Row.of(1, "Bob", 28, "San Jose", "California", 94089), + Row.of(6, "Joana", 54, "Atlanta", "Georgia", 30033)) + .build(); + + static final SourceTestStep ORDERS = + SourceTestStep.newBuilder("orders_t") + .addOption("filterable-fields", "customer_id") + .addSchema( + "order_id INT", + "customer_id INT", + "total DOUBLE", + "order_time STRING", + "proc_time AS PROCTIME()") + .producedBeforeRestore( + Row.of(1, 3, 44.44, "2020-10-10 00:00:01"), + Row.of(2, 5, 100.02, "2020-10-10 00:00:02"), + Row.of(4, 2, 92.61, "2020-10-10 00:00:04"), + Row.of(3, 1, 23.89, "2020-10-10 00:00:03"), + Row.of(6, 4, 7.65, "2020-10-10 00:00:06"), + Row.of(5, 2, 12.78, "2020-10-10 00:00:05")) + .producedAfterRestore( + Row.of(7, 6, 17.58, "2020-10-10 00:00:07"), // new customer + Row.of(9, 1, 143.21, "2020-10-10 00:00:08") // updated zip code + ) + .build(); + + static final List<String> SINK_SCHEMA = + Arrays.asList( + "order_id INT", + "total DOUBLE", + "id INT", + "name STRING", + "age INT", + "city STRING", + "state STRING", + "zipcode INT"); + + static final TableTestProgram LOOKUP_JOIN_PROJECT_PUSHDOWN = + TableTestProgram.of( + "lookup-join-project-pushdown", + "validates lookup join with project pushdown") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema( + SINK_SCHEMA.stream() + .filter(field -> !field.equals("age INT")) + .collect(Collectors.toList())) + .consumedBeforeRestore( + "+I[1, 44.44, 3, Claire, Austin, Texas, 73301]", + "+I[2, 100.02, 5, Jake, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, San Francisco, California, 95016]", + "+I[3, 23.89, 1, Bob, Mountain View, California, 94043]", + "+I[6, 7.65, 4, Shannon, Boise, Idaho, 83701]", + "+I[5, 12.78, 2, Alice, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, Atlanta, Georgia, 30033]", + "+I[9, 143.21, 1, Bob, San Jose, California, 94089]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id") + .build(); + + static final TableTestProgram LOOKUP_JOIN_FILTER_PUSHDOWN = + TableTestProgram.of( + "lookup-join-filter-pushdown", + "validates lookup join with filter pushdown") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[1, 44.44, 3, Claire, 37, Austin, Texas, 73301]", + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.age, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id AND C.age > 30") + .build(); + + static final TableTestProgram LOOKUP_JOIN_PRE_FILTER = + TableTestProgram.of("lookup-join-pre-filter", "validates lookup join with pre filter") + .setupTableSource(CUSTOMERS) + .setupTableSource(ORDERS) + .setupTableSink( + SinkTestStep.newBuilder("sink_t") + .addSchema(SINK_SCHEMA) + .consumedBeforeRestore( + "+I[2, 100.02, 5, Jake, 42, New York City, New York, 10001]", + "+I[4, 92.61, 2, Alice, 32, San Francisco, California, 95016]", + "+I[5, 12.78, 2, Alice, 32, San Francisco, California, 95016]") + .consumedAfterRestore( + "+I[7, 17.58, 6, Joana, 54, Atlanta, Georgia, 30033]") + .build()) + .runSql( + "INSERT INTO sink_t SELECT " + + "O.order_id, " + + "O.total, " + + "C.id, " + + "C.name, " + + "C.age, " + + "C.city, " + + "C.state, " + + "C.zipcode " + + "FROM orders_t as O " + + "JOIN customers_t FOR SYSTEM_TIME AS OF O.proc_time AS C " + + "ON O.customer_id = C.id AND C.age > 30 AND O.customer_id <> 3") Review Comment: Can we use a field which won't cause the filter to be pushed down to the `Orders` source? A suggestion to make the test case more different from the `LOOKUP_JOIN_FILTER_PUSHDOWN` -- 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]
