Github user wuchong commented on a diff in the pull request: https://github.com/apache/flink/pull/4266#discussion_r127153991 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeWindowInnerJoin.scala --- @@ -0,0 +1,326 @@ +/* + * 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 +import java.util.{List => JList} + +import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 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.codegen.Compiler +import org.apache.flink.table.runtime.CRowWrappingCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.slf4j.LoggerFactory + +/** + * A CoProcessFunction to support stream join stream, currently just support inner-join + * + * @param leftLowerBound + * the left stream lower bound, and -leftLowerBound is the right stream upper bound + * @param leftUpperBound + * the left stream upper bound, and -leftUpperBound is the right stream lower bound + * @param element1Type the input type of left stream + * @param element2Type the input type of right stream + * @param genJoinFuncName the function code of other non-equi condition + * @param genJoinFuncCode the function name of other non-equi condition + * + */ +class ProcTimeWindowInnerJoin( + private val leftLowerBound: Long, + private val leftUpperBound: Long, + private val element1Type: TypeInformation[Row], + private val element2Type: TypeInformation[Row], + private val genJoinFuncName: String, + private val genJoinFuncCode: String) + extends CoProcessFunction[CRow, CRow, CRow] + with Compiler[FlatJoinFunction[Row, Row, Row]]{ + + private var cRowWrapper: CRowWrappingCollector = _ + + /** other condition function **/ + private var joinFunction: FlatJoinFunction[Row, Row, Row] = _ + + /** tmp list to store expired records **/ + private var listToRemove: JList[Long] = _ + + /** state to hold left stream element **/ + private var row1MapState: MapState[Long, JList[Row]] = _ + /** state to hold right stream element **/ + private var row2MapState: MapState[Long, JList[Row]] = _ + + /** state to record last timer of left stream, 0 means no timer **/ + private var timerState1: ValueState[Long] = _ + /** state to record last timer of right stream, 0 means no timer **/ + private var timerState2: ValueState[Long] = _ + + private val leftStreamWinSize: Long = if (leftLowerBound < 0) -leftLowerBound else 0 + private val rightStreamWinSize: Long = if (leftUpperBound > 0) leftUpperBound else 0 + + val LOG = LoggerFactory.getLogger(this.getClass) + + 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() + + listToRemove = new util.ArrayList[Long]() + cRowWrapper = new CRowWrappingCollector() + cRowWrapper.setChange(true) + + // initialize row state + val rowListTypeInfo1: TypeInformation[JList[Row]] = new ListTypeInfo[Row](element1Type) + val mapStateDescriptor1: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("row1mapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo1) + row1MapState = getRuntimeContext.getMapState(mapStateDescriptor1) + + val rowListTypeInfo2: TypeInformation[JList[Row]] = new ListTypeInfo[Row](element2Type) + val mapStateDescriptor2: MapStateDescriptor[Long, JList[Row]] = + new MapStateDescriptor[Long, JList[Row]]("row2mapstate", + BasicTypeInfo.LONG_TYPE_INFO.asInstanceOf[TypeInformation[Long]], rowListTypeInfo2) + row2MapState = getRuntimeContext.getMapState(mapStateDescriptor2) + + // initialize timer state + val valueStateDescriptor1: ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("timervaluestate1", classOf[Long]) + timerState1 = getRuntimeContext.getState(valueStateDescriptor1) + + val valueStateDescriptor2: ValueStateDescriptor[Long] = + new ValueStateDescriptor[Long]("timervaluestate2", classOf[Long]) + timerState2 = getRuntimeContext.getState(valueStateDescriptor2) + } + + /** + * Process left stream records + * + * @param valueC The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement1( + valueC: CRow, + ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + out: Collector[CRow]): Unit = { + + processElement( + valueC, + ctx, + out, + leftStreamWinSize, + timerState1, + row1MapState, + row2MapState, + -leftUpperBound, // right stream lower + -leftLowerBound, // right stream upper + true + ) + } + + /** + * Process right stream records + * + * @param valueC The input value. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + * + */ + override def processElement2( + valueC: CRow, + ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + out: Collector[CRow]): Unit = { + + processElement( + valueC, + ctx, + out, + rightStreamWinSize, + timerState2, + row2MapState, + row1MapState, + leftLowerBound, // left stream upper + leftUpperBound, // left stream upper + false + ) + } + + /** + * Called when a processing timer trigger. + * Expire left/right records which earlier than current time - windowsize. + * + * @param timestamp The timestamp of the firing timer. + * @param ctx The ctx to register timer or get current time + * @param out The collector for returning result values. + */ + override def onTimer( + timestamp: Long, + ctx: CoProcessFunction[CRow, CRow, CRow]#OnTimerContext, + out: Collector[CRow]): Unit = { + + if (timerState1.value == timestamp) { + expireOutTimeRow( + timestamp, + leftStreamWinSize, + row1MapState, + timerState1, + ctx + ) + } + + if (timerState2.value == timestamp) { + expireOutTimeRow( + timestamp, + rightStreamWinSize, + row2MapState, + timerState2, + ctx + ) + } + } + + /** + * Puts an element from the input stream into state and search the other state to + * output records meet the condition, and registers a timer for the current record + * if there is no timer at present. + */ + private def processElement( + valueC: CRow, + ctx: CoProcessFunction[CRow, CRow, CRow]#Context, + out: Collector[CRow], + winSize: Long, + timerState: ValueState[Long], + rowMapState: MapState[Long, JList[Row]], + oppoRowMapState: MapState[Long, JList[Row]], + oppoLowerBound: Long, + oppoUpperBound: Long, + isLeft: Boolean): Unit = { + + cRowWrapper.out = out + + val value = valueC.row + + val curProcessTime = ctx.timerService.currentProcessingTime + val oppoLowerTime = curProcessTime + oppoLowerBound + val oppoUpperTime = curProcessTime + oppoUpperBound + + // only when windowsize != 0, we need to store the element + if (winSize != 0) { --- End diff -- I'm not sure about this. For example, `a.proctime between b.proctime - 5 and b.proctime`. In this case, we will buffer stream `a` for a window size 5, but will not buffer stream `b` because the right window size is 0. Suppose the input elements are [a1, 1], [a2, 2], [b1, 5], [a3, 5]. The first field in the tuple indicates which stream it belongs to. The seconds field in the tuple is the processing timestamp. The expected result should be `a1, b1`, `a2, b1`, `a3, b1`. But the actual result misses `a3, b1`. Because we didn't buffer the elements from `b` stream. So I think, even if the window size is 0, we still need to store the elements. Of course, we will register a `curTime +1` timer to clean the states.
--- 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 infrastruct...@apache.org or file a JIRA ticket with INFRA. ---