You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/01/20 06:46:43 UTC

[GitHub] [flink] godfreyhe commented on a change in pull request #14699: [FLINK-21011][table-planner-blink] Separate implementation of StreamExecIntervalJoin

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



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
##########
@@ -0,0 +1,352 @@
+/*
+ * 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.functions.FlatMapFunction;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.streaming.api.operators.StreamFlatMap;
+import org.apache.flink.streaming.api.operators.StreamMap;
+import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator;
+import org.apache.flink.streaming.api.transformations.OneInputTransformation;
+import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.streaming.api.transformations.UnionTransformation;
+import org.apache.flink.table.api.TableException;
+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.IntervalJoinUtil;
+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.KeyedCoProcessOperatorWithWatermarkDelay;
+import org.apache.flink.table.runtime.operators.join.OuterJoinPaddingUtil;
+import org.apache.flink.table.runtime.operators.join.interval.IntervalJoinFunction;
+import org.apache.flink.table.runtime.operators.join.interval.ProcTimeIntervalJoin;
+import org.apache.flink.table.runtime.operators.join.interval.RowTimeIntervalJoin;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** {@link StreamExecNode} for a time interval stream join. */
+public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
+        implements StreamExecNode<RowData> {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(StreamExecIntervalJoin.class);
+
+    private final JoinSpec joinSpec;
+    private final IntervalJoinUtil.WindowBounds windowBounds;
+
+    public StreamExecIntervalJoin(
+            JoinSpec joinSpec,
+            IntervalJoinUtil.WindowBounds windowBounds,

Review comment:
       port `WindowBounds` to java ?

##########
File path: flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/operators/join/interval/IntervalJoinFunction.java
##########
@@ -0,0 +1,75 @@
+package org.apache.flink.table.runtime.operators.join.interval;
+
+import org.apache.flink.api.common.functions.RichFlatJoinFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.binary.BinaryRowData;
+import org.apache.flink.table.data.utils.JoinedRowData;
+import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
+import org.apache.flink.table.runtime.generated.JoinCondition;
+import org.apache.flink.table.runtime.operators.join.NullAwareJoinHelper;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.util.Collector;
+
+/**
+ * {@link IntervalJoinFunction} is a {@link RichFlatJoinFunction} wrappers a {@link
+ * GeneratedJoinCondition}.
+ */
+public class IntervalJoinFunction extends RichFlatJoinFunction<RowData, RowData, RowData>

Review comment:
       @beyond1920 could you help to review this change ? thanks

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.physical.stream
+
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalJoin
+import org.apache.flink.table.planner.plan.nodes.exec.{ExecEdge, ExecNode}
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIntervalJoin
+import org.apache.flink.table.planner.plan.utils.IntervalJoinUtil.WindowBounds
+import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall
+import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.core.{Join, JoinRelType}
+import org.apache.calcite.rel.{RelNode, RelWriter}
+import org.apache.calcite.rex.RexNode
+
+import scala.collection.JavaConversions._
+
+/**
+  * Stream physical RelNode for a time interval stream join.
+  */
+class StreamPhysicalIntervalJoin(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    leftRel: RelNode,
+    rightRel: RelNode,
+    joinType: JoinRelType,
+    condition: RexNode,

Review comment:
       the condition in `StreamExecIntervalJoin` (the removed class)  is the whole join condition, including interval condition and remaining condition, while the condition here is only the remaining part. This change will affect explain result. 

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnUniqueness.scala
##########
@@ -501,14 +501,14 @@ class FlinkRelMdColumnUniqueness private extends MetadataHandler[BuiltInMetadata
   }
 
   def areColumnsUnique(
-      rel: StreamExecIntervalJoin,
+      rel: StreamPhysicalIntervalJoin,
       mq: RelMetadataQuery,
       columns: ImmutableBitSet,
       ignoreNulls: Boolean): JBoolean = {
-    val joinInfo = JoinInfo.of(rel.getLeft, rel.getRight, rel.joinCondition)
+    val joinInfo = JoinInfo.of(rel.getLeft, rel.getRight, rel.getCondition)

Review comment:
       use `rel.analyzeCondition()`

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdUniqueKeys.scala
##########
@@ -470,11 +470,11 @@ class FlinkRelMdUniqueKeys private extends MetadataHandler[BuiltInMetadata.Uniqu
   }
 
   def getUniqueKeys(
-      rel: StreamExecIntervalJoin,
+      rel: StreamPhysicalIntervalJoin,
       mq: RelMetadataQuery,
       ignoreNulls: Boolean): JSet[ImmutableBitSet] = {
-    val joinInfo = JoinInfo.of(rel.getLeft, rel.getRight, rel.joinCondition)
-    getJoinUniqueKeys(joinInfo, rel.joinType, rel.getLeft, rel.getRight, mq, ignoreNulls)
+    val joinInfo = JoinInfo.of(rel.getLeft, rel.getRight, rel.getCondition)

Review comment:
       use `rel.analyzeCondition()`




----------------------------------------------------------------
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:
users@infra.apache.org