godfreyhe commented on a change in pull request #14663:
URL: https://github.com/apache/flink/pull/14663#discussion_r558122588



##########
File path: 
flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
##########
@@ -0,0 +1,445 @@
+package org.apache.flink.table.planner.plan.nodes.exec.common;
+
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.streaming.api.datastream.AsyncDataStream;
+import org.apache.flink.streaming.api.functions.ProcessFunction;
+import org.apache.flink.streaming.api.functions.async.AsyncFunction;
+import org.apache.flink.streaming.api.operators.ProcessOperator;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.catalog.DataTypeFactory;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.conversion.DataStructureConverter;
+import org.apache.flink.table.data.conversion.DataStructureConverters;
+import org.apache.flink.table.functions.AsyncTableFunction;
+import org.apache.flink.table.functions.TableFunction;
+import org.apache.flink.table.functions.UserDefinedFunction;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
+import org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable;
+import org.apache.flink.table.planner.plan.schema.TableSourceTable;
+import org.apache.flink.table.planner.plan.utils.LookupJoinUtil;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.collector.TableFunctionCollector;
+import org.apache.flink.table.runtime.collector.TableFunctionResultFuture;
+import org.apache.flink.table.runtime.generated.GeneratedCollector;
+import org.apache.flink.table.runtime.generated.GeneratedFunction;
+import org.apache.flink.table.runtime.generated.GeneratedResultFuture;
+import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
+import 
org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinRunner;
+import 
org.apache.flink.table.runtime.operators.join.lookup.AsyncLookupJoinWithCalcRunner;
+import org.apache.flink.table.runtime.operators.join.lookup.LookupJoinRunner;
+import 
org.apache.flink.table.runtime.operators.join.lookup.LookupJoinWithCalcRunner;
+import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter;
+import org.apache.flink.table.runtime.typeutils.InternalSerializers;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.sources.LookupableTableSource;
+import org.apache.flink.table.sources.TableSource;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import scala.Option;
+
+/**
+ * Base {@link ExecNode} for temporal table join which shares most methods.
+ *
+ * <p>For a lookup join query:
+ *
+ * <pre>
+ * SELECT T.id, T.content, D.age
+ * FROM T JOIN userTable FOR SYSTEM_TIME AS OF T.proctime AS D
+ * ON T.content = concat(D.name, '!') AND D.age = 11 AND T.id = D.id
+ * WHERE D.name LIKE 'Jack%'
+ * </pre>
+ *
+ * <p>The LookupJoin physical node encapsulates the following RelNode tree:
+ *
+ * <pre>
+ *      Join (l.name = r.name)
+ *    /     \
+ * RelNode  Calc (concat(name, "!") as name, name LIKE 'Jack%')
+ *           |
+ *        DimTable (lookup-keys: age=11, id=l.id)
+ *     (age, id, name)
+ * </pre>
+ *
+ * <ul>
+ *   <li>lookupKeys: [$0=11, $1=l.id] ($0 and $1 is the indexes of age and id 
in dim table)
+ *   <li>calcOnTemporalTable: calc on temporal table rows before join
+ *   <li>joinCondition: join condition on temporal table rows after calc
+ * </ul>
+ *
+ * <p>The workflow of lookup join:
+ *
+ * <p>1) lookup records dimension table using the lookup-keys <br>
+ * 2) project & filter on the lookup-ed records <br>
+ * 3) join left input record and lookup-ed records <br>
+ * 4) only outputs the rows which match to the condition <br>
+ */
+public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData> {
+
+    private final FlinkJoinType joinType;
+    /**
+     * lookup keys: the key is index in dim table. the value is source of 
lookup key either
+     * constant or field from right table.
+     */
+    private final Map<Integer, LookupJoinUtil.LookupKey> lookupKeys;
+    /** the reference of temporal table to look up. */
+    private final RelOptTable temporalTable;
+    /** calc performed on rows of temporal table before join. */
+    private final Optional<RexProgram> calcOnTemporalTable;
+    /** join condition except equi-conditions extracted as lookup keys. */
+    private final Optional<RexNode> joinCondition;
+
+    protected CommonExecLookupJoin(
+            FlinkJoinType joinType,
+            Optional<RexNode> joinCondition,
+            // TODO: refactor this into TableSourceTable, once legacy 
TableSource is removed
+            RelOptTable temporalTable,

Review comment:
       extract the useful info to another class, `RelOptTable` can not be 
serialized




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

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


Reply via email to