forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
/
CommonPhysicalJoin.scala
78 lines (67 loc) · 2.98 KB
/
CommonPhysicalJoin.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
/*
* 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.common
import org.apache.flink.table.plan.FlinkJoinRelType
import org.apache.flink.table.plan.nodes.physical.FlinkPhysicalRel
import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil}
import org.apache.calcite.rel.RelWriter
import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField}
import org.apache.calcite.rel.core.{Join, SemiJoin}
import org.apache.calcite.sql.validate.SqlValidatorUtil
import org.apache.calcite.util.mapping.IntPair
import java.util
import java.util.Collections
import scala.collection.JavaConversions._
/**
* Base physical class for flink [[Join]].
*/
trait CommonPhysicalJoin extends Join with FlinkPhysicalRel {
lazy val (joinInfo, filterNulls) = {
val filterNulls = new util.ArrayList[java.lang.Boolean]
val joinInfo = FlinkRelOptUtil.createJoinInfo(getLeft, getRight, getCondition, filterNulls)
(joinInfo, filterNulls.map(_.booleanValue()).toArray)
}
lazy val keyPairs: List[IntPair] = joinInfo.pairs.toList
// TODO supports FlinkJoinRelType.ANTI
lazy val flinkJoinType: FlinkJoinRelType = this match {
case sj: SemiJoin => FlinkJoinRelType.SEMI
case j: Join => FlinkJoinRelType.toFlinkJoinRelType(getJoinType)
case _ => throw new IllegalArgumentException(s"Illegal join node: ${this.getRelTypeName}")
}
lazy val inputRowType: RelDataType = this match {
case sj: SemiJoin =>
// Combines inputs' RowType, the result is different from SemiJoin's RowType.
SqlValidatorUtil.deriveJoinRowType(
sj.getLeft.getRowType,
sj.getRight.getRowType,
getJoinType,
sj.getCluster.getTypeFactory,
null,
Collections.emptyList[RelDataTypeField]
)
case j: Join => getRowType
case _ => throw new IllegalArgumentException(s"Illegal join node: ${this.getRelTypeName}")
}
override def explainTerms(pw: RelWriter): RelWriter = {
pw.input("left", getLeft).input("right", getRight)
.item("joinType", RelExplainUtil.joinTypeToString(flinkJoinType))
.item("where",
RelExplainUtil.expressionToString(getCondition, inputRowType, getExpressionString))
.item("select", getRowType.getFieldNames.mkString(", "))
}
}