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 9E744200D23 for ; Wed, 4 Oct 2017 21:35:41 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 9CD41160BDE; Wed, 4 Oct 2017 19:35:41 +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 BE0021609D6 for ; Wed, 4 Oct 2017 21:35:40 +0200 (CEST) Received: (qmail 85461 invoked by uid 500); 4 Oct 2017 19:35:40 -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 85326 invoked by uid 99); 4 Oct 2017 19:35:39 -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; Wed, 04 Oct 2017 19:35:39 +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 4761E180DC3 for ; Wed, 4 Oct 2017 19:35:39 +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 iRlkLs8ke35u for ; Wed, 4 Oct 2017 19:35:37 +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 0F86E5FDEC for ; Wed, 4 Oct 2017 19:35:36 +0000 (UTC) Received: (qmail 84391 invoked by uid 99); 4 Oct 2017 19:35:36 -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; Wed, 04 Oct 2017 19:35:36 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7CB84F5BD1; Wed, 4 Oct 2017 19:35:36 +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 #4625: [FLINK-6233] [table] Support time-bounded stream i... Content-Type: text/plain Message-Id: <20171004193536.7CB84F5BD1@git1-us-west.apache.org> Date: Wed, 4 Oct 2017 19:35:36 +0000 (UTC) archived-at: Wed, 04 Oct 2017 19:35:41 -0000 Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4625#discussion_r142697408 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamInnerJoin.scala --- @@ -0,0 +1,410 @@ +/* + * 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.util.{ArrayList, List => JList} + +import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.ListTypeInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.table.api.Types +import org.apache.flink.table.codegen.Compiler +import org.apache.flink.table.runtime.CRowWrappingCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.table.util.Logging +import org.apache.flink.types.Row +import org.apache.flink.util.Collector + +/** + * A CoProcessFunction to execute time-bounded stream inner-join. + * Two kinds of time criteria: + * "L.time between R.time + X and R.time + Y" or "R.time between L.time - Y and L.time - X". + * + * @param leftLowerBound the lower bound for the left stream (X in the criteria) + * @param leftUpperBound the upper bound for the left stream (Y in the criteria) + * @param allowedLateness the lateness allowed for the two streams + * @param leftType the input type of left stream + * @param rightType the input type of right stream + * @param genJoinFuncName the function code of other non-equi conditions + * @param genJoinFuncCode the function name of other non-equi conditions + * + */ +abstract class TimeBoundedStreamInnerJoin( + private val leftLowerBound: Long, + private val leftUpperBound: Long, + private val allowedLateness: Long, + private val leftType: TypeInformation[Row], + private val rightType: TypeInformation[Row], + private val genJoinFuncName: String, + private val genJoinFuncCode: String, + private val leftTimeIdx: Int, + private val rightTimeIdx: Int) + extends CoProcessFunction[CRow, CRow, CRow] + with Compiler[FlatJoinFunction[Row, Row, Row]] + with Logging { + + private var cRowWrapper: CRowWrappingCollector = _ + + // the join function for other conditions + private var joinFunction: FlatJoinFunction[Row, Row, Row] = _ + + // cache to store rows from the left stream + private var leftCache: MapState[Long, JList[Row]] = _ + // cache to store rows from the right stream + private var rightCache: MapState[Long, JList[Row]] = _ + + // state to record the timer on the left stream. 0 means no timer set + private var leftTimerState: ValueState[Long] = _ + // state to record the timer on the right stream. 0 means no timer set + private var rightTimerState: ValueState[Long] = _ + + private val leftRelativeSize: Long = -leftLowerBound + private val rightRelativeSize: Long = leftUpperBound + + private var leftExpirationTime: Long = 0L; + private var rightExpirationTime: Long = 0L; + + protected var leftOperatorTime: Long = 0L + protected var rightOperatorTime: Long = 0L + + + // for delayed cleanup + private val cleanupDelay = (leftRelativeSize + rightRelativeSize) / 2 + + if (allowedLateness < 0) { + throw new IllegalArgumentException("The allowed lateness must be non-negative.") + } + + /** + * Get the maximum interval between receiving a row and emitting it (as part of a joined result). + * Only reasonable for row time join. + * + * @return the maximum delay for the outputs + */ + def getMaxOutputDelay: Long = Math.max(leftRelativeSize, rightRelativeSize) + allowedLateness + + override def open(config: Configuration) { + LOG.debug(s"Compiling JoinFunction: $genJoinFuncName \n\n " + + s"Code:\n$genJoinFuncCode") + val clazz = compile( + getRuntimeContext.getUserCodeClassLoader, + genJoinFuncName, + genJoinFuncCode) + LOG.debug("Instantiating JoinFunction.") + joinFunction = clazz.newInstance() + + cRowWrapper = new CRowWrappingCollector() + cRowWrapper.setChange(true) + + // Initialize the data caches. + val leftListTypeInfo: TypeInformation[JList[Row]] = new ListTypeInfo[Row](leftType) + val leftStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]( + "InnerJoinLeftCache", + Types.LONG.asInstanceOf[TypeInformation[Long]], + leftListTypeInfo) + leftCache = getRuntimeContext.getMapState(leftStateDescriptor) + + val rightListTypeInfo: TypeInformation[JList[Row]] = new ListTypeInfo[Row](rightType) + val rightStateDescriptor: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]( + "InnerJoinRightCache", + Types.LONG.asInstanceOf[TypeInformation[Long]], + rightListTypeInfo) + rightCache = getRuntimeContext.getMapState(rightStateDescriptor) + + // Initialize the timer states. + val leftTimerStateDesc: ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("InnerJoinLeftTimerState", classOf[Long]) + leftTimerState = getRuntimeContext.getState(leftTimerStateDesc) + + val rightTimerStateDesc: ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("InnerJoinRightTimerState", classOf[Long]) + rightTimerState = getRuntimeContext.getState(rightTimerStateDesc) + } + + /** + * Process rows from the left stream. + */ + override def processElement1( + cRowValue: CRow, + ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + out: Collector[CRow]): Unit = { + updateOperatorTime(ctx) + val leftRow = cRowValue.row + val timeForLeftRow: Long = getTimeForLeftStream(ctx, leftRow) + val rightQualifiedLowerBound: Long = timeForLeftRow - rightRelativeSize + val rightQualifiedUpperBound: Long = timeForLeftRow + leftRelativeSize + cRowWrapper.out = out + + if (rightOperatorTime < rightQualifiedUpperBound) { --- End diff -- Add comment for the branch of the condition like "Operator time of right stream has not exceeded the upper window bound of the current record. We must expect records from the right stream that join with the current record." ---