Skip to content

Commit

Permalink
[FLINK-12487] [table-planner-blink] Introduce planner rules to rewrit…
Browse files Browse the repository at this point in the history
…e expression and merge calc

This closes apache#8411
  • Loading branch information
godfreyhe authored and KurtYoung committed May 13, 2019
1 parent 76ae39a commit b47b591
Show file tree
Hide file tree
Showing 20 changed files with 1,760 additions and 152 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,7 @@ class CodeGeneratorContext(val tableConfig: TableConfig) {
}

addReusableMember(
s"final $setTypeTerm $fieldTerm = new $setTypeTerm(${elements.size})")
s"final $setTypeTerm $fieldTerm = new $setTypeTerm(${elements.size});")

elements.foreach { element =>
val content =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,11 +65,23 @@ object FlinkBatchRuleSets {
ReduceExpressionsRule.JOIN_INSTANCE
)

/**
* RuleSet to rewrite coalesce to case when
*/
private val REWRITE_COALESCE_RULES: RuleSet = RuleSets.ofList(
// rewrite coalesce to case when
RewriteCoalesceRule.FILTER_INSTANCE,
RewriteCoalesceRule.PROJECT_INSTANCE,
RewriteCoalesceRule.JOIN_INSTANCE,
RewriteCoalesceRule.CALC_INSTANCE
)

/**
* RuleSet to normalize plans for batch
*/
val DEFAULT_REWRITE_RULES: RuleSet = RuleSets.ofList((
REDUCE_EXPRESSION_RULES.asScala ++
REWRITE_COALESCE_RULES.asScala ++
REDUCE_EXPRESSION_RULES.asScala ++
List(
// Transform window to LogicalWindowAggregate
BatchLogicalWindowAggregateRule.INSTANCE,
Expand All @@ -80,7 +92,8 @@ object FlinkBatchRuleSets {
//ensure intersect set operator have the same row type
new CoerceInputsRule(classOf[LogicalIntersect], false),
//ensure except set operator have the same row type
new CoerceInputsRule(classOf[LogicalMinus], false)
new CoerceInputsRule(classOf[LogicalMinus], false),
ConvertToNotInOrInRule.INSTANCE
)).asJava)

/**
Expand Down Expand Up @@ -139,7 +152,9 @@ object FlinkBatchRuleSets {
// reorder sort and projection
ProjectSortTransposeRule.INSTANCE,
//removes constant keys from an Agg
AggregateProjectPullUpConstantsRule.INSTANCE
AggregateProjectPullUpConstantsRule.INSTANCE,
// push project through a Union
ProjectSetOpTransposeRule.INSTANCE
)

val WINDOW_RULES: RuleSet = RuleSets.ofList(
Expand Down Expand Up @@ -197,7 +212,7 @@ object FlinkBatchRuleSets {
ProjectCalcMergeRule.INSTANCE,
FilterToCalcRule.INSTANCE,
ProjectToCalcRule.INSTANCE,
CalcMergeRule.INSTANCE
FlinkCalcMergeRule.INSTANCE
)

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,11 +65,23 @@ object FlinkStreamRuleSets {
ReduceExpressionsRule.JOIN_INSTANCE
)

/**
* RuleSet to rewrite coalesce to case when
*/
private val REWRITE_COALESCE_RULES: RuleSet = RuleSets.ofList(
// rewrite coalesce to case when
RewriteCoalesceRule.FILTER_INSTANCE,
RewriteCoalesceRule.PROJECT_INSTANCE,
RewriteCoalesceRule.JOIN_INSTANCE,
RewriteCoalesceRule.CALC_INSTANCE
)

/**
* RuleSet to normalize plans for stream
*/
val DEFAULT_REWRITE_RULES: RuleSet = RuleSets.ofList((
REDUCE_EXPRESSION_RULES.asScala ++
REWRITE_COALESCE_RULES.asScala ++
REDUCE_EXPRESSION_RULES.asScala ++
List(
StreamLogicalWindowAggregateRule.INSTANCE,
// slices a project into sections which contain window agg functions
Expand All @@ -82,7 +94,8 @@ object FlinkStreamRuleSets {
//ensure intersect set operator have the same row type
new CoerceInputsRule(classOf[LogicalIntersect], false),
//ensure except set operator have the same row type
new CoerceInputsRule(classOf[LogicalMinus], false)
new CoerceInputsRule(classOf[LogicalMinus], false),
ConvertToNotInOrInRule.INSTANCE
)
).asJava)

Expand Down Expand Up @@ -187,7 +200,7 @@ object FlinkStreamRuleSets {
ProjectCalcMergeRule.INSTANCE,
FilterToCalcRule.INSTANCE,
ProjectToCalcRule.INSTANCE,
CalcMergeRule.INSTANCE
FlinkCalcMergeRule.INSTANCE
)

/**
Expand Down Expand Up @@ -230,8 +243,9 @@ object FlinkStreamRuleSets {
// transform over window to topn node
FlinkLogicalRankRule.INSTANCE,
// split distinct aggregate to reduce data skew
SplitAggregateRule.INSTANCE
// TODO add flink calc merge rule
SplitAggregateRule.INSTANCE,
// merge calc after calc transpose
FlinkCalcMergeRule.INSTANCE
)

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,187 @@
/*
* 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:https://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.rules.logical

import org.apache.flink.table.`type`.InternalTypes
import org.apache.flink.table.calcite.FlinkTypeFactory

import org.apache.calcite.plan.RelOptRule.{any, operand}
import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil}
import org.apache.calcite.rel.core.Filter
import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode}
import org.apache.calcite.sql.SqlBinaryOperator
import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, EQUALS, IN, NOT_EQUALS, NOT_IN, OR}
import org.apache.calcite.tools.RelBuilder

import scala.collection.JavaConversions._
import scala.collection.mutable

/**
* Rule for converting a cascade of predicates to [[IN]] or [[NOT_IN]].
*
* For example,
* 1. convert predicate: (x = 1 OR x = 2 OR x = 3 OR x = 4) AND y = 5
* to predicate: x IN (1, 2, 3, 4) AND y = 5.
* 2. convert predicate: (x <> 1 AND x <> 2 AND x <> 3 AND x <> 4) AND y = 5
* to predicate: x NOT IN (1, 2, 3, 4) AND y = 5.
*/
class ConvertToNotInOrInRule
extends RelOptRule(
operand(classOf[Filter], any),
"ConvertToNotInOrInRule") {

// these threshold values are set by OptimizableHashSet benchmark test on different type.
// threshold for non-float and non-double type
private val THRESHOLD: Int = 4
// threshold for float and double type
private val FRACTIONAL_THRESHOLD: Int = 20

override def onMatch(call: RelOptRuleCall): Unit = {
val filter: Filter = call.rel(0)
val condition = filter.getCondition

// convert equal expression connected by OR to IN
val inExpr = convertToNotInOrIn(call.builder(), condition, IN)
// convert not-equal expression connected by AND to NOT_IN
val notInExpr = convertToNotInOrIn(call.builder(), inExpr.getOrElse(condition), NOT_IN)

notInExpr match {
case Some(expr) =>
val newFilter = filter.copy(filter.getTraitSet, filter.getInput, expr)
call.transformTo(newFilter)
case _ =>
// check IN conversion if NOT_IN conversion is fail
inExpr match {
case Some(expr) =>
val newFilter = filter.copy(filter.getTraitSet, filter.getInput, expr)
call.transformTo(newFilter)
case _ => // do nothing
}
}
}

/**
* Returns a condition decomposed by [[AND]] or [[OR]].
*/
private def decomposedBy(rex: RexNode, operator: SqlBinaryOperator): Seq[RexNode] = {
operator match {
case AND => RelOptUtil.conjunctions(rex)
case OR => RelOptUtil.disjunctions(rex)
}
}

/**
* Convert a cascade predicates to [[IN]] or [[NOT_IN]].
*
* @param builder The [[RelBuilder]] to build the [[RexNode]].
* @param rex The predicates to be converted.
* @return The converted predicates.
*/
private def convertToNotInOrIn(
builder: RelBuilder,
rex: RexNode,
toOperator: SqlBinaryOperator): Option[RexNode] = {

// For example, when convert to [[IN]], fromOperator is [[EQUALS]].
// We convert a cascade of [[EQUALS]] to [[IN]].
// A connect operator is used to connect the fromOperator.
// A composed operator may contains sub [[IN]] or [[NOT_IN]].
val (fromOperator, connectOperator, composedOperator) = toOperator match {
case IN => (EQUALS, OR, AND)
case NOT_IN => (NOT_EQUALS, AND, OR)
}

val decomposed = decomposedBy(rex, connectOperator)
val combineMap = new mutable.HashMap[String, mutable.ListBuffer[RexCall]]
val rexBuffer = new mutable.ArrayBuffer[RexNode]
var beenConverted = false

// traverse decomposed predicates
decomposed.foreach {
case call: RexCall =>
call.getOperator match {
// put same predicates into combine map
case `fromOperator` =>
(call.operands(0), call.operands(1)) match {
case (ref, _: RexLiteral) =>
combineMap.getOrElseUpdate(ref.toString, mutable.ListBuffer[RexCall]()) += call
case (l: RexLiteral, ref) =>
combineMap.getOrElseUpdate(ref.toString, mutable.ListBuffer[RexCall]()) +=
call.clone(call.getType, List(ref, l))
case _ => rexBuffer += call
}

// process sub predicates
case `composedOperator` =>
val newRex = decomposedBy(call, composedOperator).map { r =>
convertToNotInOrIn(builder, r, toOperator) match {
case Some(ex) =>
beenConverted = true
ex
case None => r
}
}
composedOperator match {
case AND => rexBuffer += builder.and(newRex)
case OR => rexBuffer += builder.or(newRex)
}

case _ => rexBuffer += call
}

case rex => rexBuffer += rex
}

combineMap.values.foreach { list =>
if (needConvert(list.toList)) {
val inputRef = list.head.getOperands.head
val values = list.map(_.getOperands.last)
rexBuffer += builder.getRexBuilder.makeCall(toOperator, List(inputRef) ++ values)
beenConverted = true
} else {
connectOperator match {
case AND => rexBuffer += builder.and(list)
case OR => rexBuffer += builder.or(list)
}
}
}

if (beenConverted) {
// return result if has been converted
connectOperator match {
case AND => Some(builder.and(rexBuffer))
case OR => Some(builder.or(rexBuffer))
}
} else {
None
}
}

private def needConvert(rexNodes: List[RexCall]): Boolean = {
val inputRef = rexNodes.head.getOperands.head
FlinkTypeFactory.toInternalType(inputRef.getType) match {
case InternalTypes.FLOAT | InternalTypes.DOUBLE => rexNodes.size >= FRACTIONAL_THRESHOLD
case _ => rexNodes.size >= THRESHOLD
}
}
}

object ConvertToNotInOrInRule {
val INSTANCE = new ConvertToNotInOrInRule
}
Loading

0 comments on commit b47b591

Please sign in to comment.