JingsongLi commented on a change in pull request #8901: [FLINK-13003][table-planner-blink] Support Temporal TableFunction Join in blink planner URL: https://github.com/apache/flink/pull/8901#discussion_r298472910
########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/TemporalJoinUtil.scala ########## @@ -0,0 +1,112 @@ +/* + * 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.util + +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.{OperandTypes, ReturnTypes} +import org.apache.calcite.sql.{SqlFunction, SqlFunctionCategory, SqlKind} +import org.apache.flink.util.Preconditions.checkArgument + +/** + * Utilities for temporal table join + */ +object TemporalJoinUtil { + + // ---------------------------------------------------------------------------------------- + // Temporal TableFunction Join Utilities + // ---------------------------------------------------------------------------------------- + + /** + * [[TEMPORAL_JOIN_CONDITION]] is a specific condition which correctly defines + * references to rightTimeAttribute, rightPrimaryKeyExpression and leftTimeAttribute. + * The condition is used to mark this is a temporal tablefunction join. + * Later rightTimeAttribute, rightPrimaryKeyExpression and leftTimeAttribute will be + * extracted from the condition. + */ + val TEMPORAL_JOIN_CONDITION = new SqlFunction( + "__TEMPORAL_JOIN_CONDITION", + SqlKind.OTHER_FUNCTION, + ReturnTypes.BOOLEAN_NOT_NULL, + null, + OperandTypes.or( + OperandTypes.sequence( + "'(LEFT_TIME_ATTRIBUTE, RIGHT_TIME_ATTRIBUTE, PRIMARY_KEY)'", + OperandTypes.DATETIME, + OperandTypes.DATETIME, + OperandTypes.ANY), + OperandTypes.sequence( + "'(LEFT_TIME_ATTRIBUTE, PRIMARY_KEY)'", + OperandTypes.DATETIME, + OperandTypes.ANY)), + SqlFunctionCategory.SYSTEM) + + + def isRowtimeCall(call: RexCall): Boolean = { + checkArgument(call.getOperator == TEMPORAL_JOIN_CONDITION) + call.getOperands.size() == 3 + } + + def isProctimeCall(call: RexCall): Boolean = { + checkArgument(call.getOperator == TEMPORAL_JOIN_CONDITION) + call.getOperands.size() == 2 + } + + def makeRowTimeTemporalJoinConditionCall( + rexBuilder: RexBuilder, + leftTimeAttribute: RexNode, + rightTimeAttribute: RexNode, + rightPrimaryKeyExpression: RexNode): RexNode = { + rexBuilder.makeCall( + TEMPORAL_JOIN_CONDITION, + leftTimeAttribute, + rightTimeAttribute, + rightPrimaryKeyExpression) + } + + def makeProcTimeTemporalJoinConditionCall( + rexBuilder: RexBuilder, + leftTimeAttribute: RexNode, + rightPrimaryKeyExpression: RexNode): RexNode = { + rexBuilder.makeCall( + TEMPORAL_JOIN_CONDITION, + leftTimeAttribute, + rightPrimaryKeyExpression) + } + + + def containsTemporalJoinCondition(condition: RexNode): Boolean = { Review comment: separate method? I'm just suggesting that you get rid of this class. ---------------------------------------------------------------- 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