Skip to content

Commit

Permalink
[FLINK-14016][python][flink-table-planner] Introduce DataStreamPython…
Browse files Browse the repository at this point in the history
…Calc for Python function execution
  • Loading branch information
dianfu committed Sep 25, 2019
1 parent 71fa237 commit abaf8eb
Show file tree
Hide file tree
Showing 19 changed files with 944 additions and 43 deletions.
Expand Up @@ -40,6 +40,13 @@ public interface FunctionDefinition {
*/
FunctionKind getKind();

/**
* Returns the language of function this definition describes.
*/
default FunctionLanguage getLanguage() {
return FunctionLanguage.JVM;
}

/**
* Returns the set of requirements this definition demands.
*/
Expand Down
@@ -0,0 +1,32 @@
/*
* 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.functions;

import org.apache.flink.annotation.PublicEvolving;

/**
* Categorizes the language of a {@link FunctionDefinition}.
*/
@PublicEvolving
public enum FunctionLanguage {

JVM,

PYTHON
}
@@ -0,0 +1,56 @@
/*
* 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.functions.python;

import org.apache.flink.annotation.Internal;
import org.apache.flink.util.Preconditions;

/**
* A simple implementation of {@link PythonFunction}.
*/
@Internal
public final class SimplePythonFunction implements PythonFunction {

private static final long serialVersionUID = 1L;

/**
* Serialized representation of the user-defined python function.
*/
private final byte[] serializedPythonFunction;

/**
* Python execution environment.
*/
private final PythonEnv pythonEnv;

public SimplePythonFunction(byte[] serializedPythonFunction, PythonEnv pythonEnv) {
this.serializedPythonFunction = Preconditions.checkNotNull(serializedPythonFunction);
this.pythonEnv = Preconditions.checkNotNull(pythonEnv);
}

@Override
public byte[] getSerializedPythonFunction() {
return serializedPythonFunction;
}

@Override
public PythonEnv getPythonEnv() {
return pythonEnv;
}
}
Expand Up @@ -52,6 +52,12 @@ class CalciteConfigBuilder {
private var replaceLogicalOptRules: Boolean = false
private var logicalOptRuleSets: List[RuleSet] = Nil

/**
* Defines the logical optimization rule set.
*/
private var replacePythonLogicalOptRules: Boolean = false
private var pythonLogicalOptRuleSets: List[RuleSet] = Nil

/**
* Defines the physical optimization rule set.
*/
Expand Down Expand Up @@ -225,6 +231,8 @@ class CalciteConfigBuilder {
replaceNormRules,
getRuleSet(logicalOptRuleSets),
replaceLogicalOptRules,
getRuleSet(pythonLogicalOptRuleSets),
replacePythonLogicalOptRules,
getRuleSet(physicalOptRuleSets),
replacePhysicalOptRules,
getRuleSet(decoRuleSets),
Expand Down Expand Up @@ -254,6 +262,10 @@ class CalciteConfig(
val logicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in logical optimization rule set. */
val replacesLogicalOptRuleSet: Boolean,
/** A custom Python logical optimization rule set. */
val pythonLogicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in Python logical optimization rule set. */
val replacesPythonLogicalOptRuleSet: Boolean,
/** A custom physical optimization rule set. */
val physicalOptRuleSet: Option[RuleSet],
/** Whether this configuration replaces the built-in physical optimization rule set. */
Expand Down
Expand Up @@ -54,7 +54,8 @@ class BatchOptimizer(
val decorPlan = RelDecorrelator.decorrelateQuery(expandedPlan)
val normalizedPlan = optimizeNormalizeLogicalPlan(decorPlan)
val logicalPlan = optimizeLogicalPlan(normalizedPlan)
optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASET)
val pythonizedLogicalPlan = optimizePythonLogicalPlan(logicalPlan)
optimizePhysicalPlan(pythonizedLogicalPlan, FlinkConventions.DATASET)
}

/**
Expand Down
Expand Up @@ -86,6 +86,25 @@ abstract class Optimizer(
}
}

/**
* Returns the logical Python optimization rule set for this optimizer
* including a custom RuleSet configuration.
*/
protected def getPythonLogicalOptRuleSet: RuleSet = {
materializedConfig.pythonLogicalOptRuleSet match {

case None =>
getBuiltInPythonLogicalOptRuleSet

case Some(ruleSet) =>
if (materializedConfig.replacesLogicalOptRuleSet) {
ruleSet
} else {
RuleSets.ofList((getBuiltInPythonLogicalOptRuleSet.asScala ++ ruleSet.asScala).asJava)
}
}
}

/**
* Returns the physical optimization rule set for this optimizer
* including a custom RuleSet configuration.
Expand Down Expand Up @@ -117,6 +136,13 @@ abstract class Optimizer(
FlinkRuleSets.LOGICAL_OPT_RULES
}

/**
* Returns the built-in Python logical optimization rules that are defined by the optimizer.
*/
protected def getBuiltInPythonLogicalOptRuleSet: RuleSet = {
FlinkRuleSets.LOGICAL_PYTHON_OPT_RULES
}

/**
* Returns the built-in physical optimization rules that are defined by the optimizer.
*/
Expand Down Expand Up @@ -153,6 +179,19 @@ abstract class Optimizer(
}
}

protected def optimizePythonLogicalPlan(relNode: RelNode): RelNode = {
val logicalOptRuleSet = getPythonLogicalOptRuleSet
if (logicalOptRuleSet.iterator().hasNext) {
runHepPlannerSimultaneously(
HepMatchOrder.TOP_DOWN,
logicalOptRuleSet,
relNode,
relNode.getTraitSet)
} else {
relNode
}
}

protected def optimizeLogicalPlan(relNode: RelNode): RelNode = {
val logicalOptRuleSet = getLogicalOptRuleSet
val logicalOutputProps = relNode.getTraitSet.replace(FlinkConventions.LOGICAL).simplify()
Expand Down
Expand Up @@ -64,8 +64,8 @@ class StreamOptimizer(
RelTimeIndicatorConverter.convert(decorPlan, relBuilder.getRexBuilder)
val normalizedPlan = optimizeNormalizeLogicalPlan(planWithMaterializedTimeAttributes)
val logicalPlan = optimizeLogicalPlan(normalizedPlan)

val physicalPlan = optimizePhysicalPlan(logicalPlan, FlinkConventions.DATASTREAM)
val pythonizedLogicalPlan = optimizePythonLogicalPlan(logicalPlan)
val physicalPlan = optimizePhysicalPlan(pythonizedLogicalPlan, FlinkConventions.DATASTREAM)
optimizeDecoratePlan(physicalPlan, updatesAsRetraction)
}

Expand Down
Expand Up @@ -34,7 +34,6 @@ trait CommonCalc {
private[flink] def generateFunction[T <: Function](
generator: FunctionCodeGenerator,
ruleDescription: String,
inputSchema: RowSchema,
returnSchema: RowSchema,
calcProjection: Seq[RexNode],
calcCondition: Option[RexNode],
Expand Down
@@ -0,0 +1,72 @@
/*
* 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

import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode}
import org.apache.flink.table.functions.FunctionLanguage
import org.apache.flink.table.functions.python.{PythonFunction, PythonFunctionInfo, SimplePythonFunction}
import org.apache.flink.table.functions.utils.ScalarSqlFunction

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

trait CommonPythonCalc {

private[flink] def extractPythonScalarFunctionInfos(
rexCalls: Array[RexCall]): (Array[Int], Array[PythonFunctionInfo]) = {
// using LinkedHashMap to keep the insert order
val inputNodes = new mutable.LinkedHashMap[RexNode, Integer]()
val pythonFunctionInfos = rexCalls.map(createPythonScalarFunctionInfo(_, inputNodes))

val udfInputOffsets = inputNodes.toArray.sortBy(_._2).map(_._1).map {
case inputRef: RexInputRef => inputRef.getIndex
}
(udfInputOffsets, pythonFunctionInfos)
}

private[flink] def createPythonScalarFunctionInfo(
rexCall: RexCall,
inputNodes: mutable.Map[RexNode, Integer]): PythonFunctionInfo = rexCall.getOperator match {
case sfc: ScalarSqlFunction if sfc.getScalarFunction.getLanguage == FunctionLanguage.PYTHON =>
val inputs = new mutable.ArrayBuffer[AnyRef]()
rexCall.getOperands.foreach {
case pythonRexCall: RexCall if pythonRexCall.getOperator.asInstanceOf[ScalarSqlFunction]
.getScalarFunction.getLanguage == FunctionLanguage.PYTHON =>
// Continuous Python UDFs can be chained together
val argPythonInfo = createPythonScalarFunctionInfo(pythonRexCall, inputNodes)
inputs.append(argPythonInfo)

case argNode: RexNode =>
// For input arguments of RexInputRef, it's replaced with an offset into the input row;
inputNodes.get(argNode) match {
case Some(existing) => inputs.append(existing)
case None =>
val inputOffset = Integer.valueOf(inputNodes.size)
inputs.append(inputOffset)
inputNodes.put(argNode, inputOffset)
}
}

// Extracts the necessary information for Python function execution, such as
// the serialized Python function, the Python env, etc
val pythonFunction = new SimplePythonFunction(
sfc.getScalarFunction.asInstanceOf[PythonFunction].getSerializedPythonFunction,
sfc.getScalarFunction.asInstanceOf[PythonFunction].getPythonEnv)
new PythonFunctionInfo(pythonFunction, inputs.toArray)
}
}
Expand Up @@ -105,7 +105,6 @@ class DataSetCalc(
val genFunction = generateFunction(
generator,
ruleDescription,
new RowSchema(getInput.getRowType),
new RowSchema(getRowType),
projection,
condition,
Expand Down
Expand Up @@ -18,18 +18,15 @@

package org.apache.flink.table.plan.nodes.datastream

import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet}
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.core.Calc
import org.apache.calcite.rel.metadata.RelMetadataQuery
import org.apache.calcite.rel.{RelNode, RelWriter}
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rex.RexProgram
import org.apache.flink.streaming.api.datastream.DataStream
import org.apache.flink.streaming.api.functions.ProcessFunction
import org.apache.flink.table.api.StreamQueryConfig
import org.apache.flink.table.calcite.RelTimeIndicatorConverter
import org.apache.flink.table.codegen.FunctionCodeGenerator
import org.apache.flink.table.plan.nodes.CommonCalc
import org.apache.flink.table.plan.schema.RowSchema
import org.apache.flink.table.planner.StreamPlanner
import org.apache.flink.table.runtime.CRowProcessRunner
Expand All @@ -49,11 +46,14 @@ class DataStreamCalc(
schema: RowSchema,
calcProgram: RexProgram,
ruleDescription: String)
extends Calc(cluster, traitSet, input, calcProgram)
with CommonCalc
with DataStreamRel {

override def deriveRowType(): RelDataType = schema.relDataType
extends DataStreamCalcBase(
cluster,
traitSet,
input,
inputSchema,
schema,
calcProgram,
ruleDescription) {

override def copy(traitSet: RelTraitSet, child: RelNode, program: RexProgram): Calc = {
new DataStreamCalc(
Expand All @@ -66,28 +66,6 @@ class DataStreamCalc(
ruleDescription)
}

override def toString: String = calcToString(calcProgram, getExpressionString)

override def explainTerms(pw: RelWriter): RelWriter = {
pw.input("input", getInput)
.item("select", selectionToString(calcProgram, getExpressionString))
.itemIf("where",
conditionToString(calcProgram, getExpressionString),
calcProgram.getCondition != null)
}

override def computeSelfCost(planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = {
val child = this.getInput
val rowCnt = metadata.getRowCount(child)
computeSelfCost(calcProgram, planner, rowCnt)
}

override def estimateRowCount(metadata: RelMetadataQuery): Double = {
val child = this.getInput
val rowCnt = metadata.getRowCount(child)
estimateRowCount(calcProgram, rowCnt)
}

override def translateToPlan(
planner: StreamPlanner,
queryConfig: StreamQueryConfig): DataStream[CRow] = {
Expand Down Expand Up @@ -117,7 +95,6 @@ class DataStreamCalc(
val genFunction = generateFunction(
generator,
ruleDescription,
inputSchema,
schema,
projection,
condition,
Expand Down

0 comments on commit abaf8eb

Please sign in to comment.