Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-12 Thread via GitHub


dawidwys merged PR #23814:
URL: https://github.com/apache/flink/pull/23814


-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-08 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1421106534


##
flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-lookup-join_1/lookup-join-filter-pushdown/plan/lookup-join-filter-pushdown.json:
##
@@ -0,0 +1,264 @@
+{
+  "flinkVersion" : "1.19",
+  "nodes" : [ {
+"id" : 5,
+"type" : "stream-exec-table-source-scan_1",
+"scanTableSource" : {
+  "table" : {
+"identifier" : "`default_catalog`.`default_database`.`orders_t`",
+"resolvedTable" : {
+  "schema" : {
+"columns" : [ {
+  "name" : "order_id",
+  "dataType" : "INT"
+}, {
+  "name" : "customer_id",
+  "dataType" : "INT"
+}, {
+  "name" : "total",
+  "dataType" : "DOUBLE"
+}, {
+  "name" : "order_time",
+  "dataType" : "VARCHAR(2147483647)"
+}, {
+  "name" : "proc_time",
+  "kind" : "COMPUTED",
+  "expression" : {
+"rexNode" : {
+  "kind" : "CALL",
+  "internalName" : "$PROCTIME$1",
+  "operands" : [ ],
+  "type" : {
+"type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE",
+"nullable" : false,
+"precision" : 3,
+"kind" : "PROCTIME"
+  }
+},
+"serializableString" : "PROCTIME()"
+  }
+} ],
+"watermarkSpecs" : [ ]
+  },
+  "partitionKeys" : [ ]
+}
+  },
+  "abilities" : [ {
+"type" : "ProjectPushDown",
+"projectedFields" : [ [ 0 ], [ 1 ], [ 2 ] ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  }, {
+"type" : "ReadingMetadata",
+"metadataKeys" : [ ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  } ]
+},
+"outputType" : "ROW<`order_id` INT, `customer_id` INT, `total` DOUBLE>",
+"description" : "TableSourceScan(table=[[default_catalog, 
default_database, orders_t, project=[order_id, customer_id, total], 
metadata=[]]], fields=[order_id, customer_id, total])",
+"inputProperties" : [ ]
+  }, {
+"id" : 6,
+"type" : "stream-exec-lookup-join_1",
+"joinType" : "INNER",
+"joinCondition" : null,
+"temporalTable" : {
+  "lookupTableSource" : {
+"table" : {
+  "identifier" : "`default_catalog`.`default_database`.`customers_t`",
+  "resolvedTable" : {
+"schema" : {
+  "columns" : [ {
+"name" : "id",
+"dataType" : "INT NOT NULL"
+  }, {
+"name" : "name",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "age",
+"dataType" : "INT"
+  }, {
+"name" : "city",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "state",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "zipcode",
+"dataType" : "INT"
+  } ],
+  "watermarkSpecs" : [ ],
+  "primaryKey" : {
+"name" : "PK_3386",

Review Comment:
   Fixed. Thanks @jnh5y 



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-08 Thread via GitHub


jnh5y commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1421076042


##
flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-lookup-join_1/lookup-join-filter-pushdown/plan/lookup-join-filter-pushdown.json:
##
@@ -0,0 +1,264 @@
+{
+  "flinkVersion" : "1.19",
+  "nodes" : [ {
+"id" : 5,
+"type" : "stream-exec-table-source-scan_1",
+"scanTableSource" : {
+  "table" : {
+"identifier" : "`default_catalog`.`default_database`.`orders_t`",
+"resolvedTable" : {
+  "schema" : {
+"columns" : [ {
+  "name" : "order_id",
+  "dataType" : "INT"
+}, {
+  "name" : "customer_id",
+  "dataType" : "INT"
+}, {
+  "name" : "total",
+  "dataType" : "DOUBLE"
+}, {
+  "name" : "order_time",
+  "dataType" : "VARCHAR(2147483647)"
+}, {
+  "name" : "proc_time",
+  "kind" : "COMPUTED",
+  "expression" : {
+"rexNode" : {
+  "kind" : "CALL",
+  "internalName" : "$PROCTIME$1",
+  "operands" : [ ],
+  "type" : {
+"type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE",
+"nullable" : false,
+"precision" : 3,
+"kind" : "PROCTIME"
+  }
+},
+"serializableString" : "PROCTIME()"
+  }
+} ],
+"watermarkSpecs" : [ ]
+  },
+  "partitionKeys" : [ ]
+}
+  },
+  "abilities" : [ {
+"type" : "ProjectPushDown",
+"projectedFields" : [ [ 0 ], [ 1 ], [ 2 ] ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  }, {
+"type" : "ReadingMetadata",
+"metadataKeys" : [ ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  } ]
+},
+"outputType" : "ROW<`order_id` INT, `customer_id` INT, `total` DOUBLE>",
+"description" : "TableSourceScan(table=[[default_catalog, 
default_database, orders_t, project=[order_id, customer_id, total], 
metadata=[]]], fields=[order_id, customer_id, total])",
+"inputProperties" : [ ]
+  }, {
+"id" : 6,
+"type" : "stream-exec-lookup-join_1",
+"joinType" : "INNER",
+"joinCondition" : null,
+"temporalTable" : {
+  "lookupTableSource" : {
+"table" : {
+  "identifier" : "`default_catalog`.`default_database`.`customers_t`",
+  "resolvedTable" : {
+"schema" : {
+  "columns" : [ {
+"name" : "id",
+"dataType" : "INT NOT NULL"
+  }, {
+"name" : "name",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "age",
+"dataType" : "INT"
+  }, {
+"name" : "city",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "state",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "zipcode",
+"dataType" : "INT"
+  } ],
+  "watermarkSpecs" : [ ],
+  "primaryKey" : {
+"name" : "PK_3386",

Review Comment:
   @bvarghese1 The compiled plans need to be recomputed so that they pick up 
the new identifier scheme!
   
   This will fix the present CI failure.



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-08 Thread via GitHub


jnh5y commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1421076042


##
flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-lookup-join_1/lookup-join-filter-pushdown/plan/lookup-join-filter-pushdown.json:
##
@@ -0,0 +1,264 @@
+{
+  "flinkVersion" : "1.19",
+  "nodes" : [ {
+"id" : 5,
+"type" : "stream-exec-table-source-scan_1",
+"scanTableSource" : {
+  "table" : {
+"identifier" : "`default_catalog`.`default_database`.`orders_t`",
+"resolvedTable" : {
+  "schema" : {
+"columns" : [ {
+  "name" : "order_id",
+  "dataType" : "INT"
+}, {
+  "name" : "customer_id",
+  "dataType" : "INT"
+}, {
+  "name" : "total",
+  "dataType" : "DOUBLE"
+}, {
+  "name" : "order_time",
+  "dataType" : "VARCHAR(2147483647)"
+}, {
+  "name" : "proc_time",
+  "kind" : "COMPUTED",
+  "expression" : {
+"rexNode" : {
+  "kind" : "CALL",
+  "internalName" : "$PROCTIME$1",
+  "operands" : [ ],
+  "type" : {
+"type" : "TIMESTAMP_WITH_LOCAL_TIME_ZONE",
+"nullable" : false,
+"precision" : 3,
+"kind" : "PROCTIME"
+  }
+},
+"serializableString" : "PROCTIME()"
+  }
+} ],
+"watermarkSpecs" : [ ]
+  },
+  "partitionKeys" : [ ]
+}
+  },
+  "abilities" : [ {
+"type" : "ProjectPushDown",
+"projectedFields" : [ [ 0 ], [ 1 ], [ 2 ] ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  }, {
+"type" : "ReadingMetadata",
+"metadataKeys" : [ ],
+"producedType" : "ROW<`order_id` INT, `customer_id` INT, `total` 
DOUBLE> NOT NULL"
+  } ]
+},
+"outputType" : "ROW<`order_id` INT, `customer_id` INT, `total` DOUBLE>",
+"description" : "TableSourceScan(table=[[default_catalog, 
default_database, orders_t, project=[order_id, customer_id, total], 
metadata=[]]], fields=[order_id, customer_id, total])",
+"inputProperties" : [ ]
+  }, {
+"id" : 6,
+"type" : "stream-exec-lookup-join_1",
+"joinType" : "INNER",
+"joinCondition" : null,
+"temporalTable" : {
+  "lookupTableSource" : {
+"table" : {
+  "identifier" : "`default_catalog`.`default_database`.`customers_t`",
+  "resolvedTable" : {
+"schema" : {
+  "columns" : [ {
+"name" : "id",
+"dataType" : "INT NOT NULL"
+  }, {
+"name" : "name",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "age",
+"dataType" : "INT"
+  }, {
+"name" : "city",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "state",
+"dataType" : "VARCHAR(2147483647)"
+  }, {
+"name" : "zipcode",
+"dataType" : "INT"
+  } ],
+  "watermarkSpecs" : [ ],
+  "primaryKey" : {
+"name" : "PK_3386",

Review Comment:
   @bvarghese1 The compiled plans need to be recomputed so that they pick up 
the new identifier scheme!



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-07 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1419886852


##
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"
-

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-07 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1419886732


##
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"
-

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-07 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1419461565


##
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 
extraOptions) {
 final Map allOptions = new HashMap<>(options);
-allOptions.putAll(extraOptions);

Review Comment:
   Fixed



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-07 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1419461234


##
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:
   Fixed



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-03 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1413375613


##
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 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")
+  

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-03 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1413375404


##
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 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")
+  

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-03 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1413375281


##
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:
   I think its fine. I have updated the comment to clarify this.



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-12-03 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1413356579


##
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"
-

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-30 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1411425256


##
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 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")
+  

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-30 Thread via GitHub


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


##
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 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")
+

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-29 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1409648619


##
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 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")
+  

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-29 Thread via GitHub


bvarghese1 commented on code in PR #23814:
URL: https://github.com/apache/flink/pull/23814#discussion_r1409647014


##
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 
extraOptions) {
 final Map allOptions = new HashMap<>(options);
-allOptions.putAll(extraOptions);

Review Comment:
   Alright. Will do that



-- 
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



Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-29 Thread via GitHub


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 i

Re: [PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-27 Thread via GitHub


flinkbot commented on PR #23814:
URL: https://github.com/apache/flink/pull/23814#issuecomment-1828441266

   
   ## CI report:
   
   * 40058dbde8aa6676a411809110590296bd40a47c UNKNOWN
   
   
   Bot commands
 The @flinkbot bot supports the following commands:
   
- `@flinkbot run azure` re-run the last Azure build
   


-- 
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



[PR] [FLINK-33647] Implement restore tests for LookupJoin node [flink]

2023-11-27 Thread via GitHub


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

   
   
   ## What is the purpose of the change
   
   Implement restore tests for LookupJoin node
   
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
 - Added restore tests for LookupJoin 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-unsubscr...@flink.apache.org

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