From issues-return-155516-archive-asf-public=cust-asf.ponee.io@flink.apache.org Mon Feb 26 16:09:07 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 81623180671 for ; Mon, 26 Feb 2018 16:09:06 +0100 (CET) Received: (qmail 11464 invoked by uid 500); 26 Feb 2018 15:09:05 -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 11382 invoked by uid 99); 26 Feb 2018 15:09:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 26 Feb 2018 15:09:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 09AA8C6064 for ; Mon, 26 Feb 2018 15:09:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.311 X-Spam-Level: X-Spam-Status: No, score=-110.311 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id GuIDX2a33-_C for ; Mon, 26 Feb 2018 15:09:03 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 12B905F613 for ; Mon, 26 Feb 2018 15:09:03 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 1C692E0956 for ; Mon, 26 Feb 2018 15:09:02 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 3EC4E255DC for ; Mon, 26 Feb 2018 15:09:01 +0000 (UTC) Date: Mon, 26 Feb 2018 15:09:01 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-8428) Implement stream-stream non-window left outer join MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-8428?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1637= 7006#comment-16377006 ]=20 ASF GitHub Bot commented on FLINK-8428: --------------------------------------- Github user walterddr commented on a diff in the pull request: https://github.com/apache/flink/pull/5327#discussion_r170617462 =20 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/t= able/runtime/join/NonWindowLeftJoinWithNonEquiPredicates.scala --- @@ -0,0 +1,228 @@ +/* + * 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 imp= lied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.flink.table.runtime.join + +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.tuple.{Tuple2 =3D> JTuple2} +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.table.api.{StreamQueryConfig, Types} +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.apache.flink.util.Collector + +/** + * Connect data for left stream and right stream. Only use for LeftJo= in with NonEquiPredicates. + * An MapState of type [Row, Long] is added to record how many rows f= rom the right table can be + * matched for each left row. Left join without NonEquiPredicates doe= sn't need it because + * left rows can always join right rows as long as join keys are same= . + * + * @param leftType the input type of left stream + * @param rightType the input type of right stream + * @param resultType the output type of join + * @param genJoinFuncName the function code of other non-equi conditi= on + * @param genJoinFuncCode the function name of other non-equi conditi= on + * @param queryConfig the configuration for the query to generate + */ +class NonWindowLeftJoinWithNonEquiPredicates( + leftType: TypeInformation[Row], + rightType: TypeInformation[Row], + resultType: TypeInformation[CRow], + genJoinFuncName: String, + genJoinFuncCode: String, + queryConfig: StreamQueryConfig) + extends NonWindowJoin( + leftType, + rightType, + resultType, + genJoinFuncName, + genJoinFuncCode, + queryConfig) { + + // result row, all field from right will be null + private var resultRow: Row =3D _ + // how many matched rows from the right table for each left row + private var leftJoinCnt: MapState[Row, Long] =3D _ + + override def open(parameters: Configuration): Unit =3D { + super.open(parameters) + + val leftJoinCntDescriptor =3D new MapStateDescriptor[Row, Long]( + "leftJoinCnt", leftType, Types.LONG.asInstanceOf[TypeInformation= [Long]]) + leftJoinCnt =3D getRuntimeContext.getMapState(leftJoinCntDescripto= r) + resultRow =3D new Row(resultType.getArity) + + LOG.debug("Instantiating NonWindowLeftJoinWithNonEquiPredicates.") + } + + /** + * Puts or Retract an element from the input stream into state and = search the other state to + * output records meet the condition. The result is NULL from the r= ight side, if there is no + * match. Records will be expired in state if state retention time = has been specified. + */ + override def processElement( + value: CRow, + ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + out: Collector[CRow], + timerState: ValueState[Long], + currentSideState: MapState[Row, JTuple2[Int, Long]], + otherSideState: MapState[Row, JTuple2[Int, Long]], + isLeft: Boolean): Unit =3D { + + val inputRow =3D value.row + cRowWrapper.reset() + cRowWrapper.setCollector(out) + cRowWrapper.setChange(value.change) + + val curProcessTime =3D ctx.timerService.currentProcessingTime + val oldCntAndExpiredTime =3D currentSideState.get(inputRow) + val cntAndExpiredTime =3D if (null =3D=3D oldCntAndExpiredTime) { + JTuple2.of(0, -1L) + } else { + oldCntAndExpiredTime + } + + cntAndExpiredTime.f1 =3D getNewExpiredTime(curProcessTime, cntAndE= xpiredTime.f1) + if (stateCleaningEnabled && timerState.value() =3D=3D 0) { + timerState.update(cntAndExpiredTime.f1) + ctx.timerService().registerProcessingTimeTimer(cntAndExpiredTime= .f1) + } + + // update current side stream state + if (!value.change) { + cntAndExpiredTime.f0 =3D cntAndExpiredTime.f0 - 1 + if (cntAndExpiredTime.f0 <=3D 0) { + currentSideState.remove(inputRow) + if (isLeft) { + leftJoinCnt.remove(inputRow) + } + } else { + currentSideState.put(inputRow, cntAndExpiredTime) + } + } else { + cntAndExpiredTime.f0 =3D cntAndExpiredTime.f0 + 1 + currentSideState.put(inputRow, cntAndExpiredTime) + } + val otherSideIterator =3D otherSideState.iterator() + cRowWrapper.setEmitCnt(0) + // join other side data + if (isLeft) { + while (otherSideIterator.hasNext) { + val otherSideEntry =3D otherSideIterator.next() + val otherSideRow =3D otherSideEntry.getKey + val cntAndExpiredTimeOfOtherSide =3D otherSideEntry.getValue + // join + cRowWrapper.setTimes(cntAndExpiredTimeOfOtherSide.f0) + joinFunction.join(inputRow, otherSideRow, cRowWrapper) + // clear expired data. Note: clear after join to keep closer t= o the original semantics + if (stateCleaningEnabled && curProcessTime >=3D cntAndExpiredT= imeOfOtherSide.f1) { + otherSideIterator.remove() + } + } + // update matched cnt only when left row cnt is changed from 0 t= o 1. Each time encountered a + // new record from right, leftJoinCnt will also be updated. + if (cntAndExpiredTime.f0 =3D=3D 1) { --- End diff -- =20 Can't this check be triggered if `cntAndExpiredTime` was updated from 2= =3D=3D> 1? > Implement stream-stream non-window left outer join > -------------------------------------------------- > > Key: FLINK-8428 > URL: https://issues.apache.org/jira/browse/FLINK-8428 > Project: Flink > Issue Type: Sub-task > Components: Table API & SQL > Reporter: Hequn Cheng > Assignee: Hequn Cheng > Priority: Major > > Implement stream-stream non-window left outer join for sql/table-api. A s= imple design doc can be found=C2=A0[here|https://docs.google.com/document/d= /1u7bJHeEBP_hFhi8Jm4oT3FqQDOm2pJDqCtq1U1WMHDo/edit?usp=3Dsharing] -- This message was sent by Atlassian JIRA (v7.6.3#76005)