Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3715#discussion_r115850396
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala
 ---
    @@ -0,0 +1,180 @@
    +/*
    + * 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.plan.nodes.datastream
    +
    +import org.apache.calcite.plan._
    +import org.apache.calcite.rel.core.JoinRelType
    +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
    +import org.apache.flink.api.common.functions.RichFilterFunction
    +import org.apache.flink.api.java.functions.NullByteKeySelector
    +import org.apache.flink.streaming.api.datastream.DataStream
    +import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
    +import org.apache.flink.table.calcite.FlinkTypeFactory
    +import org.apache.flink.table.plan.nodes.CommonJoin
    +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin}
    +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
    +import org.apache.flink.types.Row
    +
    +/**
    +  * Flink RelNode which matches along with JoinOperator and its related 
operations.
    +  */
    +class DataStreamJoin(
    +   cluster: RelOptCluster,
    +   traitSet: RelTraitSet,
    +   leftNode: RelNode,
    +   rightNode: RelNode,
    +   joinNode: FlinkLogicalJoin,
    +   leftSchema: RowSchema,
    +   schema: RowSchema,
    +   ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +  with CommonJoin
    +  with DataStreamRel {
    +
    +  override def deriveRowType() = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: 
java.util.List[RelNode]): RelNode = {
    +    new DataStreamJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinNode,
    +      leftSchema,
    +      schema,
    +      ruleDescription)
    +  }
    +
    +  override def toString: String = {
    +
    +    s"${joinTypeToString(joinNode.getJoinType)}" +
    +      s"(condition: (${joinConditionToString(schema.logicalType,
    +        joinNode.getCondition, getExpressionString)}), " +
    +      s"select: (${joinSelectionToString(schema.logicalType)}))"
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    super.explainTerms(pw)
    +      .item("condition", joinConditionToString(schema.logicalType,
    +        joinNode.getCondition, getExpressionString))
    +      .item("select", joinSelectionToString(schema.logicalType))
    +      .item("joinType", joinTypeToString(joinNode.getJoinType))
    +  }
    +
    +  override def translateToPlan(tableEnv: StreamTableEnvironment): 
DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +
    +    // get the equality keys and other condition
    +    val (leftKeys, rightKeys, otherCondition) =
    +      JoinUtil.analyzeJoinCondition(joinNode, getExpressionString)
    --- End diff --
    
    This can be done by 
    ```
    val joinInfo = JoinInfo.of(leftNode, rightNode, condition)
    val leftKeys: Array[Int] = joinInfo.leftKeys.toIntArray
    val rightKeys: Array[Int] = joinInfo.rightKeys.toIntArray
    val otherCondition = joinInfo.getRemaining(cluster.getRexBuilder)
    ``` 
    So we do not need a special method for this. The type checks are not 
required, because Calcite will make sure during validation that only compatible 
types are compared. So we can be sure that types are valid.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to