godfreyhe commented on a change in pull request #14685: URL: https://github.com/apache/flink/pull/14685#discussion_r559882945
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java ########## @@ -0,0 +1,159 @@ +/* + * 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.dag.Transformation; +import org.apache.flink.streaming.api.transformations.TwoInputTransformation; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.data.RowData; +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.nodes.exec.utils.JoinSpec; +import org.apache.flink.table.planner.plan.utils.JoinUtil; +import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; +import org.apache.flink.table.runtime.generated.GeneratedJoinCondition; +import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.stream.AbstractStreamingJoinOperator; +import org.apache.flink.table.runtime.operators.join.stream.StreamingJoinOperator; +import org.apache.flink.table.runtime.operators.join.stream.StreamingSemiAntiJoinOperator; +import org.apache.flink.table.runtime.operators.join.stream.state.JoinInputSideSpec; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Lists; + +import java.util.List; + +/** + * {@link StreamExecNode} for regular Joins. + * + * <p>Regular joins are the most generic type of join in which any new records or changes to either + * side of the join input are visible and are affecting the whole join result. + */ +public class StreamExecJoin extends ExecNodeBase<RowData> implements StreamExecNode<RowData> { + + private final JoinSpec joinSpec; + private final List<int[]> leftUniqueKeys; + private final List<int[]> rightUniqueKeys; + + public StreamExecJoin( + JoinSpec joinSpec, + List<int[]> leftUniqueKeys, + List<int[]> rightUniqueKeys, + ExecEdge leftEdge, + ExecEdge rightEdge, + RowType outputType, + String description) { + super(Lists.newArrayList(leftEdge, rightEdge), outputType, description); + this.joinSpec = joinSpec; + this.leftUniqueKeys = leftUniqueKeys; + this.rightUniqueKeys = rightUniqueKeys; + } + + @Override + @SuppressWarnings("unchecked") + protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) { + final TableConfig tableConfig = planner.getTableConfig(); + + final ExecNode<RowData> leftInput = (ExecNode<RowData>) getInputNodes().get(0); + final ExecNode<RowData> rightInput = (ExecNode<RowData>) getInputNodes().get(1); + + final Transformation<RowData> leftTransform = leftInput.translateToPlan(planner); + final Transformation<RowData> rightTransform = rightInput.translateToPlan(planner); + + final RowType leftType = (RowType) leftInput.getOutputType(); + final RowType rightType = (RowType) rightInput.getOutputType(); + JoinUtil.validateJoinSpec(joinSpec, leftType, rightType, true); + + final int[] leftJoinKey = joinSpec.getLeftKeys(); + final int[] rightJoinKey = joinSpec.getRightKeys(); + + final InternalTypeInfo<RowData> leftTypeInfo = InternalTypeInfo.of(leftType); + final JoinInputSideSpec leftInputSpec = + JoinUtil.analyzeJoinInput(leftTypeInfo, leftJoinKey, leftUniqueKeys); + + final InternalTypeInfo<RowData> rightTypeInfo = InternalTypeInfo.of(leftType); + final JoinInputSideSpec rightInputSpec = + JoinUtil.analyzeJoinInput(rightTypeInfo, rightJoinKey, rightUniqueKeys); + + RowDataKeySelector leftSelect = + KeySelectorUtil.getRowDataSelector(leftJoinKey, leftTypeInfo); + RowDataKeySelector rightSelect = + KeySelectorUtil.getRowDataSelector(rightJoinKey, rightTypeInfo); Review comment: nit: move them closed to where it is used ---------------------------------------------------------------- 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: us...@infra.apache.org