1.JoinConditionEqualityTransferRule
Flink扩展的规则,是做相等条件传递的,目前只在批执行模式下有应用
l_a = r_b and l_a = r_c的条件推导出 r_b = r_c,同时l_a = r_b和l_a = r_c只会保留一个
1.1.Operand
匹配条件很简单,就是匹配Join
operand(classOf[Join], any)
1.2.matches
进一步的匹配条件。这里有两个条件:1、只支持INNER和SEMI类型;2、列的数目要大于2
第二条很好理解,根据规则的应用场景,需要至少3个列存在;第一条,理论上,left join可以推导右子树,right join可以推导左子树,场景缺失还是有其他规则补充?
if (joinType != JoinRelType.INNER && joinType != JoinRelType.SEMI) {
return false
}
val (optimizableFilters, _) = partitionJoinFilters(join)
val groups = getEquiFilterRelationshipGroup(optimizableFilters)
groups.exists(_.size > 2)
1.3.onMatch
整个过程:1、取分区连接条件(即左右子树列相等条件);2、取相等条件里的列;3、构建新的相等条件;4、构建新的JoinFilter;5、构建新的join
1.3.1.partitionJoinFilters
获取等值条件的方法
首先使用RelOptUtil.conjunctions进行条件切分,切分的原理比较简单:遇到AND就递归切分,不是AND就成为一个独立条件
if (rexPredicate.isA(SqlKind.AND)) {
for (RexNode operand : ((RexCall) rexPredicate).getOperands()) {
decomposeConjunction(operand, rexList);
}
} else {
rexList.add(rexPredicate);
}
之后,遍历条件,选出是相等条件且分别来自左右子树列的那种条件,放入一个列表,剩余的放入另一个列表
判断是否在左子树是使用的列的index进行判断的,flink(Calcite)里列的下标是使用$1这种形式展现的,join就是按从左到右的顺序标的
private def fromJoinLeft(join: Join, ref: RexInputRef): Boolean = {
require(join.getSystemFieldList.size() == 0)
ref.getIndex < join.getLeft.getRowType.getFieldCount
}
1.3.2.getEquiFilterRelationshipGroup
把条件(满足相等且列分别来自左右子树的)里的列全部取出来,这个就是一个简单的RexCall结果解析和Set去重
1.3.3.newEquiJoinFilters
构建新的相等条件时,会将前面取得的列分成左右两个列表,然后分别在makeCalls里尝试构建相等条件
相等条件的构建也很简单,就是拿第一个列跟后面其他的列进行一个相等条件的构建,注意不会形成两两的相等关系,而是以一个列为中心,其他都跟它相等
if (nodes.length > 1) {
val rex = nodes.head
nodes.drop(1).foreach(calls += rexBuilder.makeCall(EQUALS, rex, _))
}
左右构建完以后,还会使用左边一个列和右边一个列构建一个相等关系(这一块应该是因为前面的相等关系都不用了,左右子树的相等连接关系已经不存在的原因,这个规则里,完全弃用了旧的条件组,生成了一个新的条件组)
1.3.4.newJoinFilter
这一段是使用前面生成的新条件(newEquiJoinFilters),然后加上之前获取条件时不符合场景的条件(partitionJoinFilters里的第二个列表)合在一起,组成一组新的条件
这里会进行一个条件简化,就是比如a = b AND b = a简化成a = b这样。这里依赖的还是Calcite提供的工具类
1.3.5.newJoin
生成新的join节点
这里就是利用的join类的copy方法,直接基于旧的join内容和新生成的newJoinFilter产生一个新的join节点,然后用transformTo声明替换
2.本地Debug
Flink的本地Debug依赖LocalStreamEnvironment,表可以基于createTemporaryView由DataStream生成
2.1.pom依赖
因为要本地运行Flink服务,所以会需要依赖比较多的服务端包,还有提交作业的client包
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>1.15.2</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>1.15.2</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-planner_2.12</artifactId>
<version>1.15.2</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime</artifactId>
<version>1.15.2</version>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients</artifactId>
<version>1.15.2</version>
</dependency>
2.2.Debug代码
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import java.util.Arrays;
public class ListTable {
public static void main(String[] args) {
LocalStreamEnvironment env = LocalStreamEnvironment.createLocalEnvironment();
env.setRuntimeMode(RuntimeExecutionMode.BATCH);// 部分规则只有batch有
StreamTableEnvironment bsTableEnv = StreamTableEnvironment.create(env);
final DataStream<Order> orderA =
env.fromCollection(Arrays.asList(
new Order(1, 1,11),
new Order(2, 22,22),
new Order(2, 2,2),
new Order(3, 333,33)));
final DataStream<Order> orderB =
env.fromCollection(Arrays.asList(
new Order(1, 1,11),
new Order(2, 22,22),
new Order(2, 2,2),
new Order(4, 444,44)));
bsTableEnv.createTemporaryView("t1", orderA);
bsTableEnv.createTemporaryView("t2", orderB);
String sql = "SELECT * FROM t1 join t2 ON t1.id = t2.id and t1.id = t2.score and t1.id = t2.tid";
bsTableEnv.executeSql(sql).print();
}
public static class Order {
public int tid;
public int id;
public int score;
// sql语句获取列必须的
public Order(){}
// for structured type detection in Table API
public Order(int id, int tid, int score) {
this.tid = tid;
this.id = id;
this.score = score;
}
}
}
3.Debug
3.1.optimizableFilters
满足规则条件的优化列表
3.2.equiFiltersToOpt
满足规则的Filter条件里的列集合
3.3.新Filter条件
基于右子树生成的新条件如下,可以看到,并不是两两相等,而是取出Set里的第一列和其他的列进行相等;左子树由于只有一个列,没有新条件生成;最后还加上了一个左右子树某列相等的条件(就是取出各自set的第一个进行相等连接)
3.4.newJoin
最终的join可以看到,只有condition条件变了(id也变了,这是唯一标识)
旧的join结构
新的join结构