Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 86C6A200CBB for ; Tue, 4 Jul 2017 16:29:56 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8549F1614DD; Tue, 4 Jul 2017 14:29:56 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7F3371614DE for ; Tue, 4 Jul 2017 16:29:55 +0200 (CEST) Received: (qmail 34478 invoked by uid 500); 4 Jul 2017 14:29:54 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 34469 invoked by uid 99); 4 Jul 2017 14:29:54 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Jul 2017 14:29:54 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 446E318040F for ; Tue, 4 Jul 2017 14:29:54 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.021 X-Spam-Level: X-Spam-Status: No, score=-4.021 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id 2Q7OrTZqRT71 for ; Tue, 4 Jul 2017 14:29:52 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id A2AFA5FBDF for ; Tue, 4 Jul 2017 14:29:51 +0000 (UTC) Received: (qmail 33903 invoked by uid 99); 4 Jul 2017 14:29:50 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Jul 2017 14:29:50 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 943AEE381C; Tue, 4 Jul 2017 14:29:50 +0000 (UTC) From: fhueske To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3715: [FLINK-6232][Table&SQL]Support proctime inner equi... Content-Type: text/plain Message-Id: <20170704142950.943AEE381C@git1-us-west.apache.org> Date: Tue, 4 Jul 2017 14:29:50 +0000 (UTC) archived-at: Tue, 04 Jul 2017 14:29:56 -0000 Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3715#discussion_r125458518 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/WindowJoinUtil.scala --- @@ -0,0 +1,361 @@ +/* + * 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.runtime.join + +import java.math.{BigDecimal => JBigDecimal} +import java.util + +import org.apache.calcite.plan.RelOptUtil +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.JoinRelType +import org.apache.calcite.rex._ +import org.apache.calcite.sql.SqlKind +import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.table.api.{TableConfig, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.codegen.{CodeGenerator, ExpressionReducer} +import org.apache.flink.table.functions.TimeMaterializationSqlFunction +import org.apache.flink.table.plan.schema.{RowSchema, TimeIndicatorRelDataType} +import org.apache.flink.types.Row + +import scala.collection.JavaConversions._ + +/** + * An util class to help analyze and build join code . + */ +object WindowJoinUtil { + + /** + * Analyze time-condtion to get time boundary for each stream and get the time type + * and return remain condition. + * + * @param condition join condition + * @param leftLogicalFieldCnt left stream logical field num + * @param leftPhysicalFieldCnt left stream physical field num + * @param inputType left and right connect stream type + * @param rexBuilder util to build rexNode + * @param config table environment config + * @return timetype, left lower boundary, right lower boundary, remain condition + */ + private[flink] def analyzeTimeBoundary( + condition: RexNode, + leftLogicalFieldCnt: Int, + leftPhysicalFieldCnt: Int, + inputType: RelDataType, + rexBuilder: RexBuilder, + config: TableConfig): (RelDataType, Long, Long, Option[RexNode]) = { + + // Converts the condition to conjunctive normal form (CNF) + val cnfCondition = RexUtil.toCnf(rexBuilder, condition) + + // split the condition into time indicator condition and other condition + val (timeTerms, remainTerms) = + splitJoinCondition( + cnfCondition, + leftLogicalFieldCnt, + inputType + ) + + if (timeTerms.size != 2) { + throw new TableException("A time-based stream join requires exactly " + + "two join predicates that bound the time in both directions.") + } + + // extract time offset from the time indicator conditon + val streamTimeOffsets = + timeTerms.map(x => extractTimeOffsetFromCondition(x._3, x._2, rexBuilder, config)) + + val (leftLowerBound, leftUpperBound) = + streamTimeOffsets match { + case Seq((x, true), (y, false)) => (x, y) + case Seq((x, false), (y, true)) => (y, x) + case _ => + throw new TableException( + "Time-based join conditions must reference the time attribute of both input tables.") + } + + // compose the remain condition list into one condition + val remainCondition = + remainTerms match { + case Seq() => None + case _ => + // turn the logical field index to physical field index + def transInputRef(expr: RexNode): RexNode = { + expr match { + case c: RexCall => + val newOps = c.operands.map(transInputRef(_)) + rexBuilder.makeCall(c.getType, c.getOperator, newOps) + case i: RexInputRef if i.getIndex >= leftLogicalFieldCnt => + rexBuilder.makeInputRef( + i.getType, + i.getIndex - leftLogicalFieldCnt + leftPhysicalFieldCnt) + case _ => expr + } + } + + Some(remainTerms.map(transInputRef(_)).reduceLeft( (l, r) => { + RelOptUtil.andJoinFilters(rexBuilder, l, r) + })) + } + + (timeTerms.get(0)._1, leftLowerBound, leftUpperBound, remainCondition) + } + + /** + * Split the join conditions into time condition and non-time condition + * + * @return (Seq(timeTerms), Seq(remainTerms)), + */ + private def splitJoinCondition( + cnfCondition: RexNode, + leftFieldCount: Int, + inputType: RelDataType): (Seq[(RelDataType, Boolean, RexNode)], Seq[RexNode]) = { + + cnfCondition match { + case c: RexCall if c.getKind == SqlKind.AND => + val timeIndicators = + c.operands.map(splitJoinCondition(_, leftFieldCount, inputType)) + timeIndicators.reduceLeft { (l, r) => + (l._1 ++ r._1, l._2 ++ r._2) + } + case c: RexCall => + val timeIndicators = extractTimeIndicatorAccesses(c, leftFieldCount, inputType) + timeIndicators match { + case Seq() => + (Seq(), Seq(c)) + case Seq(v1, v2) => + if (v1._1 != v2._1) { + throw new TableException( + "Both time attributes in a join condition must be of the same type.") + } + if (v1._2 == v2._2) { + throw new TableException("Time-based join conditions " + + "must reference the time attribute of both input tables.") + } + (Seq((v1._1, v1._2, c)), Seq()) + case _ => + throw new TableException( + "Time-based join conditions must reference the time attribute of both input tables.") + } + case other => + val timeIndicators = extractTimeIndicatorAccesses(other, leftFieldCount, inputType) + timeIndicators match { + case Seq() => + (Seq(), Seq(other)) + case _ => + throw new TableException("Time indicators can not be used in non time-condition.") + } + } + } + + /** + * analysis if condition term has time indicator + * + * @return seq(timeType, is left input time indicator) + */ + def extractTimeIndicatorAccesses( + expression: RexNode, + leftFieldCount: Int, + inputType: RelDataType): Seq[(RelDataType, Boolean)] = { + + expression match { + case i: RexInputRef => + val idx = i.getIndex + inputType.getFieldList.get(idx).getType match { + case t: TimeIndicatorRelDataType if idx < leftFieldCount => + // left table time indicator + Seq((t, true)) + case t: TimeIndicatorRelDataType => + // right table time indicator + Seq((t, false)) + case _ => Seq() + } + case c: RexCall => + c.operands.map(extractTimeIndicatorAccesses(_, leftFieldCount, inputType)).reduce(_++_) + case _ => Seq() + } + } + + /** + * Extract time offset and determain it's the lower bound of left stream or the upper bound + * + * @return window boundary, is left lower bound + */ + def extractTimeOffsetFromCondition( + timeTerm: RexNode, + isLeftExprBelongLeftTable: Boolean, + rexBuilder: RexBuilder, + config: TableConfig): (Long, Boolean) = { + + val timeCall: RexCall = timeTerm.asInstanceOf[RexCall] + + val isLeftLowerBound: Boolean = + timeTerm.getKind match { + // e.g a.proctime > b.proctime - 5 sec, then it's the lower bound of a and the value is -5 + // e.g b.proctime > a.proctime - 5 sec, then it's not the lower bound of a but upper bound + case kind @ (SqlKind.GREATER_THAN | SqlKind.GREATER_THAN_OR_EQUAL) => + isLeftExprBelongLeftTable + // e.g a.proctime < b.proctime + 5 sec, the the upper bound of a is 5 + case kind @ (SqlKind.LESS_THAN | SqlKind.LESS_THAN_OR_EQUAL) => + !isLeftExprBelongLeftTable + case _ => + throw new TableException("Unsupport time-condition.") + } + + val (leftLiteral, rightLiteral) = + reduceTimeExpression( + timeCall.operands.get(0), + timeCall.operands.get(1), + rexBuilder, + config) + val tmpTimeOffset: Long = + if (isLeftExprBelongLeftTable) rightLiteral - leftLiteral else leftLiteral - rightLiteral + + val boundary = + tmpTimeOffset.signum * ( + if (timeTerm.getKind == SqlKind.LESS_THAN || timeTerm.getKind == SqlKind.GREATER_THAN) { + tmpTimeOffset.abs - 1 + } else { + tmpTimeOffset.abs + }) + + (boundary, isLeftLowerBound) + } + + /** + * Calcute the time boundary. Replace the rowtime/proctime with zero literal. + * For example: + * b.proctime - interval '1' second - interval '2' second will be translate to + * 0 - 1000 - 2000 + */ + private def reduceTimeExpression( + leftRexNode: RexNode, + rightRexNode: RexNode, + rexBuilder: RexBuilder, + config: TableConfig): (Long, Long) = { + + /** + * replace the rowtime/proctime with zero literal. + */ + def replaceTimeFieldWithLiteral(expr: RexNode): RexNode = { + expr match { + case c: RexCall if c.getOperator == TimeMaterializationSqlFunction => --- End diff -- This case should not happen anymore because we removed the time materialization for join conditions. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastructure@apache.org or file a JIRA ticket with INFRA. ---