[ 
https://issues.apache.org/jira/browse/FLINK-6149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15972454#comment-15972454
 ] 

ASF GitHub Bot commented on FLINK-6149:
---------------------------------------

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

    https://github.com/apache/flink/pull/3594#discussion_r111917910
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/rel/logical/FlinkLogicalJoin.scala
 ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.rel.logical
    +
    +import org.apache.calcite.plan._
    +import org.apache.calcite.plan.volcano.RelSubset
    +import org.apache.calcite.rel.RelNode
    +import org.apache.calcite.rel.convert.ConverterRule
    +import org.apache.calcite.rel.core._
    +import org.apache.calcite.rel.logical.LogicalJoin
    +import org.apache.calcite.rel.metadata.RelMetadataQuery
    +import org.apache.calcite.rex.RexNode
    +import org.apache.flink.table.rel.FlinkConventions
    +
    +import scala.collection.JavaConverters._
    +
    +class FlinkLogicalJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    left: RelNode,
    +    right: RelNode,
    +    condition: RexNode,
    +    joinType: JoinRelType)
    +  extends Join(cluster, traitSet, left, right, condition, 
Set.empty[CorrelationId].asJava, joinType)
    +  with FlinkLogicalRel {
    +
    +  override def copy(
    +      traitSet: RelTraitSet,
    +      conditionExpr: RexNode,
    +      left: RelNode,
    +      right: RelNode,
    +      joinType: JoinRelType,
    +      semiJoinDone: Boolean): Join = {
    +
    +    new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, 
joinType)
    +  }
    +
    +  override def computeSelfCost (planner: RelOptPlanner, metadata: 
RelMetadataQuery): RelOptCost = {
    +    val leftRowCnt = metadata.getRowCount(getLeft)
    +    val leftRowSize = estimateRowSize(getLeft.getRowType)
    +
    +    val rightRowCnt = metadata.getRowCount(getRight)
    +    val rightRowSize = estimateRowSize(getRight.getRowType)
    +
    +    val ioCost = (leftRowCnt * leftRowSize) + (rightRowCnt * rightRowSize)
    +    val cpuCost = leftRowCnt + rightRowCnt
    +    val rowCnt = leftRowCnt + rightRowCnt
    +
    +    planner.getCostFactory.makeCost(rowCnt, cpuCost, ioCost)
    +  }
    +}
    +
    +private class FlinkLogicalJoinConverter
    +  extends ConverterRule(
    +    classOf[LogicalJoin],
    +    Convention.NONE,
    +    FlinkConventions.LOGICAL,
    +    "FlinkLogicalJoinConverter") {
    +
    +  override def matches(call: RelOptRuleCall): Boolean = {
    +    val join: LogicalJoin = call.rel(0).asInstanceOf[LogicalJoin]
    +    val joinInfo = join.analyzeCondition
    +
    +    hasEqualityPredicates(join, joinInfo) || isSingleRowInnerJoin(join)
    --- End diff --
    
    I see, thanks for the explanation. I agree, we need to keep the restriction 
here, to push the logical plan in the right direction. 
    
    We might need different sets of logical optimization rules for batch and 
streaming at some point.


> add additional flink logical relation nodes
> -------------------------------------------
>
>                 Key: FLINK-6149
>                 URL: https://issues.apache.org/jira/browse/FLINK-6149
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Kurt Young
>            Assignee: Kurt Young
>




--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to