KurtYoung commented on a change in pull request #8302: [FLINK-12269][table-blink] Support Temporal Table Join in blink planner and runtime URL: https://github.com/apache/flink/pull/8302#discussion_r281893667
########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala ########## @@ -0,0 +1,731 @@ +/* + * 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 com.google.common.primitives.Primitives +import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField} +import org.apache.calcite.rel.core.{JoinInfo, JoinRelType} +import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} +import org.apache.calcite.rex._ +import org.apache.calcite.sql.SqlKind +import org.apache.calcite.sql.fun.SqlStdOperatorTable +import org.apache.calcite.sql.validate.SqlValidatorUtil +import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.mapping.IntPair +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.{RowTypeInfo, TypeExtractor} +import org.apache.flink.streaming.api.datastream.AsyncDataStream.OutputMode +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment +import org.apache.flink.streaming.api.functions.async.ResultFuture +import org.apache.flink.streaming.api.operators.ProcessOperator +import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator +import org.apache.flink.streaming.api.transformations.{OneInputTransformation, StreamTransformation} +import org.apache.flink.table.`type`._ +import org.apache.flink.table.api.{TableConfig, TableException, TableSchema} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.LookupJoinCodeGenerator._ +import org.apache.flink.table.codegen.{CodeGeneratorContext, LookupJoinCodeGenerator} +import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{getParamClassesConsiderVarArgs, getUserDefinedMethod, signatureToString, signaturesToString} +import org.apache.flink.table.functions.{AsyncTableFunction, TableFunction, UserDefinedFunction} +import org.apache.flink.table.plan.nodes.FlinkRelNode +import org.apache.flink.table.plan.schema.TimeIndicatorRelDataType +import org.apache.flink.table.plan.util.{JoinTypeUtil, RelExplainUtil} +import org.apache.flink.table.plan.util.LookupJoinUtil._ +import org.apache.flink.table.runtime.join.lookup.{AsyncLookupJoinRunner, LookupJoinRunner, AsyncLookupJoinWithCalcRunner, LookupJoinWithCalcRunner} +import org.apache.flink.table.sources.TableIndex.IndexType +import org.apache.flink.table.sources.{LookupConfig, LookupableTableSource, TableIndex, TableSource} +import org.apache.flink.table.typeutils.BaseRowTypeInfo +import org.apache.flink.types.Row + +import java.util.Collections + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +/** + * Common abstract RelNode for temporal table join which shares most methods. + * @param input input rel node + * @param tableSource the table source to be temporal joined + * @param tableRowType the row type of the table source + * @param calcOnTemporalTable the calc (projection&filter) after table scan before joining + */ +abstract class CommonLookupJoin( + cluster: RelOptCluster, + traitSet: RelTraitSet, + input: RelNode, + val tableSource: TableSource[_], + tableRowType: RelDataType, + val calcOnTemporalTable: Option[RexProgram], + val joinInfo: JoinInfo, + val joinType: JoinRelType) + extends SingleRel(cluster, traitSet, input) + with FlinkRelNode { + + val joinKeyPairs: Array[IntPair] = getTemporalTableJoinKeyPairs(joinInfo, calcOnTemporalTable) + val indexKeys: Array[TableIndex] = getTableIndexes(tableSource) + // all potential index keys, mapping from field index in table source to LookupKey + val allLookupKeys: Map[Int, LookupKey] = analyzeLookupKeys( + cluster.getRexBuilder, + joinKeyPairs, + indexKeys, + tableSource.getTableSchema, + calcOnTemporalTable) + // the matched best lookup fields which is in defined order, maybe empty + val matchedLookupFields: Option[Array[Int]] = findMatchedIndex( + indexKeys, + tableSource.getTableSchema, + allLookupKeys) + + override def deriveRowType(): RelDataType = { + val flinkTypeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] + val rightType = if (calcOnTemporalTable.isDefined) { + calcOnTemporalTable.get.getOutputRowType + } else { + tableRowType + } + SqlValidatorUtil.deriveJoinRowType( + input.getRowType, + rightType, + joinType, + flinkTypeFactory, + null, + Collections.emptyList[RelDataTypeField]) + } + + override def explainTerms(pw: RelWriter): RelWriter = { + val remaining = joinInfo.getRemaining(cluster.getRexBuilder) + val joinCondition = if (remaining.isAlwaysTrue) { + None + } else { + Some(remaining) + } + + val inputFieldNames = input.getRowType.getFieldNames.asScala.toArray + val tableFieldNames = tableSource.getTableSchema.getFieldNames + val rightFieldNames = calcOnTemporalTable match { + case Some(calc) => calc.getOutputRowType.getFieldNames.asScala.toArray + case None => tableFieldNames + } + val resultFieldNames = getRowType.getFieldNames.asScala.toArray + val lookupConfig = getLookupConfig(tableSource.asInstanceOf[LookupableTableSource[_]]) + val whereString = calcOnTemporalTable match { + case Some(calc) => RelExplainUtil.conditionToString(calc, getExpressionString) + case None => "N/A" + } + + super.explainTerms(pw) + .item("table", tableSource.explainSource()) + .item("joinType", JoinTypeUtil.toFlinkJoinType(joinType)) + .item("async", lookupConfig.isAsyncEnabled) + .item("on", joinOnToString(inputFieldNames, rightFieldNames, joinInfo)) + .itemIf("where", whereString, calcOnTemporalTable.isDefined) + .itemIf("joinCondition", + joinConditionToString(resultFieldNames, joinCondition), + joinCondition.isDefined) + .item("select", joinSelectionToString(getRowType)) + } + + // ---------------------------------------------------------------------------------------- + // Physical Translation + // ---------------------------------------------------------------------------------------- + + def translateToPlanInternal( + inputTransformation: StreamTransformation[BaseRow], + env: StreamExecutionEnvironment, + config: TableConfig, + relBuilder: RelBuilder): StreamTransformation[BaseRow] = { + + val inputRowType = FlinkTypeFactory.toInternalRowType(input.getRowType) + val tableSourceRowType = FlinkTypeFactory.toInternalRowType(tableRowType) + val resultRowType = FlinkTypeFactory.toInternalRowType(getRowType) + val tableSchema = tableSource.getTableSchema + + // validate whether the node is valid and supported. + validate( + tableSource, + inputRowType, + tableSourceRowType, + indexKeys, + allLookupKeys, + matchedLookupFields, + joinType) + + val lookupFieldsInOrder = matchedLookupFields.get + val lookupFieldNamesInOrder = lookupFieldsInOrder.map(tableSchema.getFieldNames()(_)) + val lookupFieldTypesInOrder = lookupFieldsInOrder + .map(tableSchema.getFieldTypes()(_)) + .map(TypeConverters.createInternalTypeFromTypeInfo) + val remainingCondition = getRemainingJoinCondition( + cluster.getRexBuilder, + relBuilder, + input.getRowType, + tableRowType, + calcOnTemporalTable, + lookupFieldsInOrder, + joinKeyPairs, + joinInfo, + allLookupKeys) + + val lookupableTableSource = tableSource.asInstanceOf[LookupableTableSource[_]] + val lookupConfig = getLookupConfig(lookupableTableSource) + val leftOuterJoin = joinType == JoinRelType.LEFT + + val operator = if (lookupConfig.isAsyncEnabled) { + val asyncBufferCapacity= lookupConfig.getAsyncBufferCapacity + val asyncTimeout = lookupConfig.getAsyncTimeoutMs + + val asyncLookupFunction = lookupableTableSource + .getAsyncLookupFunction(lookupFieldNamesInOrder) + // return type valid check + val udtfResultType = asyncLookupFunction.getResultType + val extractedResultTypeInfo = TypeExtractor.createTypeInfo( + asyncLookupFunction, + classOf[AsyncTableFunction[_]], + asyncLookupFunction.getClass, + 0) + checkUdtfReturnType( + tableSource.explainSource(), + tableSource.getReturnType, + udtfResultType, + extractedResultTypeInfo) + val parameters = Array(new GenericType(classOf[ResultFuture[_]])) ++ lookupFieldTypesInOrder + checkEvalMethodSignature( + asyncLookupFunction, + parameters, + extractedResultTypeInfo) + + val generatedFetcher = LookupJoinCodeGenerator.generateAsyncLookupFunction( + config, + relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory], + inputRowType, + resultRowType, + tableSource.getReturnType, + lookupFieldsInOrder, + allLookupKeys, + asyncLookupFunction) + + val asyncFunc = if (calcOnTemporalTable.isDefined) { + // a projection or filter after table source scan + val rightRowType = FlinkTypeFactory + .toInternalRowType(calcOnTemporalTable.get.getOutputRowType) + val generatedResultFuture = LookupJoinCodeGenerator.generateTableAsyncCollector( + config, + "TableFunctionResultFuture", + inputRowType, + rightRowType, + remainingCondition) + val generatedCalc = generateCalcMapFunction( + config, + calcOnTemporalTable, + tableSourceRowType) + + new AsyncLookupJoinWithCalcRunner( + generatedFetcher, + generatedCalc, + generatedResultFuture, + tableSource.getReturnType, + rightRowType.toTypeInfo, + leftOuterJoin, + lookupConfig.getAsyncBufferCapacity) + } else { + // right type is the same as table source row type, because no calc after temporal table + val rightRowType = tableSourceRowType + val generatedResultFuture = LookupJoinCodeGenerator.generateTableAsyncCollector( + config, + "TableFunctionResultFuture", + inputRowType, + rightRowType, + remainingCondition) + new AsyncLookupJoinRunner( + generatedFetcher, + generatedResultFuture, + tableSource.getReturnType, + rightRowType.toTypeInfo, + leftOuterJoin, + asyncBufferCapacity) + } + + // force ORDERED output mode currently, optimize it to UNORDERED + // when the downstream do not need orderness + new AsyncWaitOperator(asyncFunc, asyncTimeout, asyncBufferCapacity, OutputMode.ORDERED) + } else { + // sync join + val lookupFunction = lookupableTableSource.getLookupFunction(lookupFieldNamesInOrder) + // return type valid check + val udtfResultType = lookupFunction.getResultType + val extractedResultTypeInfo = TypeExtractor.createTypeInfo( + lookupFunction, + classOf[TableFunction[_]], + lookupFunction.getClass, + 0) + checkUdtfReturnType( + tableSource.explainSource(), + tableSource.getReturnType, + udtfResultType, + extractedResultTypeInfo) + checkEvalMethodSignature( + lookupFunction, + lookupFieldTypesInOrder, + extractedResultTypeInfo) + + val generatedFetcher = LookupJoinCodeGenerator.generateLookupFunction( + config, + relBuilder.getTypeFactory.asInstanceOf[FlinkTypeFactory], + inputRowType, + resultRowType, + tableSource.getReturnType, + lookupFieldsInOrder, + allLookupKeys, + lookupFunction, + env.getConfig.isObjectReuseEnabled) + + val ctx = CodeGeneratorContext(config) + val processFunc = if (calcOnTemporalTable.isDefined) { + // a projection or filter after table source scan + val rightRowType = FlinkTypeFactory + .toInternalRowType(calcOnTemporalTable.get.getOutputRowType) + val generatedCollector = generateCollector( + ctx, + inputRowType, + rightRowType, + resultRowType, + remainingCondition, + None) + val generatedCalc = generateCalcMapFunction( + config, + calcOnTemporalTable, + tableSourceRowType) + + new LookupJoinWithCalcRunner( + generatedFetcher, + generatedCalc, + generatedCollector, + leftOuterJoin, + rightRowType.getArity) + } else { + // right type is the same as table source row type, because no calc after temporal table + val rightRowType = tableSourceRowType + val generatedCollector = generateCollector( + ctx, + inputRowType, + rightRowType, + resultRowType, + remainingCondition, + None) + new LookupJoinRunner( + generatedFetcher, + generatedCollector, + leftOuterJoin, + rightRowType.getArity) + } + new ProcessOperator(processFunc) + } + + new OneInputTransformation( + inputTransformation, + "TemporalTableJoin", Review comment: change the name here ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services