beyond1920 commented on a change in pull request #15195:
URL: https://github.com/apache/flink/pull/15195#discussion_r594057275



##########
File path: 
flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowJoinUtil.scala
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.utils
+
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery
+import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin
+import org.apache.flink.table.planner.plan.`trait`.RelWindowProperties
+
+import org.apache.calcite.rex.{RexInputRef, RexNode, RexUtil}
+import org.apache.calcite.sql.fun.SqlStdOperatorTable
+import org.apache.calcite.util.ImmutableIntList
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+
+/**
+ * Util for window join.
+ */
+object WindowJoinUtil {
+
+  /**
+   * Get window properties of left and right child.
+   *
+   * @param join input join
+   * @return window properties of left and right child.
+   */
+  def getChildWindowProperties(
+      join: FlinkLogicalJoin): (RelWindowProperties, RelWindowProperties) = {
+    val fmq = 
FlinkRelMetadataQuery.reuseOrCreate(join.getCluster.getMetadataQuery)
+    (fmq.getRelWindowProperties(join.getLeft), 
fmq.getRelWindowProperties(join.getRight))
+  }
+
+  /**
+   * Checks whether join condition contains window starts equality of input 
tables or window
+   * ends equality of input tables.
+   *
+   * @param join input join
+   * @return True if join condition contains window starts equality of input 
tables and window
+   *         ends equality of input tables. Else false.
+   */
+  def containsWindowStartEqualityOrEndEquality(join: FlinkLogicalJoin): 
Boolean = {
+    val (windowStartEqualityLeftKeys, windowEndEqualityLeftKeys, _, _) =
+      excludeWindowStartEqualityAndEndEqualityFromJoinInfoPairs(join)
+    windowStartEqualityLeftKeys.nonEmpty || windowEndEqualityLeftKeys.nonEmpty
+  }
+
+  /**
+   * Excludes window starts equality and window ends equality from join info.
+   *
+   * @param join input join
+   * @return Remain join information after excludes window starts equality and 
window ends
+   *         equality from join.
+   *         The first element is left join keys of window starts equality,
+   *         the second element is left join keys of window ends equality,
+   *         the third element is right join keys of window starts equality,
+   *         the forth element is right join keys of window ends equality,
+   *         the fifth element is remain left join keys,
+   *         the sixth element is remain right join keys,
+   *         the last element is remain join condition which includes remain 
equal condition and
+   *         non-equal condition.
+   */
+  def excludeWindowStartEqualityAndEndEqualityFromJoinCondition(
+      join: FlinkLogicalJoin): (
+    ImmutableIntList,
+    ImmutableIntList,
+    ImmutableIntList,
+    ImmutableIntList,
+    ImmutableIntList,
+    ImmutableIntList,
+    RexNode) = {
+    val (
+      windowStartEqualityLeftKeys,
+      windowEndEqualityLeftKeys,
+      windowStartEqualityRightKeys,
+      windowEndEqualityRightKeys) =
+      excludeWindowStartEqualityAndEndEqualityFromJoinInfoPairs(join)
+
+    val joinInfo = join.analyzeCondition()
+    val (remainLeftKeys, remainRightKeys, remainCondition) = if (
+      windowStartEqualityLeftKeys.nonEmpty || 
windowEndEqualityLeftKeys.nonEmpty) {
+      val joinFieldsType = join.getRowType.getFieldList
+      val leftFieldCnt = join.getLeft.getRowType.getFieldCount
+      val rexBuilder = join.getCluster.getRexBuilder
+      val remainEquals = mutable.ArrayBuffer[RexNode]()
+      val remainLeftKeysArray = mutable.ArrayBuffer[Int]()
+      val remainRightKeysArray = mutable.ArrayBuffer[Int]()
+      // convert remain pairs to RexInputRef tuple for building 
SqlStdOperatorTable.EQUALS calls
+      joinInfo.pairs().foreach { p =>
+        if (!windowStartEqualityLeftKeys.contains(p.source) &&
+          !windowEndEqualityLeftKeys.contains(p.source)) {
+          val leftFieldType = joinFieldsType.get(p.source).getType
+          val leftInputRef = new RexInputRef(p.source, leftFieldType)
+          val rightIndex = leftFieldCnt + p.target
+          val rightFieldType = joinFieldsType.get(rightIndex).getType
+          val rightInputRef = new RexInputRef(rightIndex, rightFieldType)
+          val remainEqual = rexBuilder.makeCall(
+            SqlStdOperatorTable.EQUALS,

Review comment:
       @wuchong Thanks, the solution you mentioned works well of course. 
However I prefer let `StreamPhysicalWindowJoin` extends `CommonPhysicalJoin`, 
so a complete condition which contains equal condition and non-equal condition 
is required, JoinInfo would be inferred in the `Join` node. What do you think?




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


Reply via email to