Github user twalthr commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4471#discussion_r152773431
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala
 ---
    @@ -0,0 +1,285 @@
    +/*
    + * 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 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.TupleTypeInfo
    +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.CRowWrappingMultiOuputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +import org.slf4j.LoggerFactory
    +import org.apache.flink.table.codegen.Compiler
    +
    +
    +/**
    +  * Connect data for left stream and right stream. Only use for innerJoin.
    +  *
    +  * @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 condition
    +  * @param genJoinFuncCode   the function name of other non-equi condition
    +  * @param queryConfig       the configuration for the query to generate
    +  */
    +class DataStreamInnerJoin(
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    resultType: TypeInformation[CRow],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String,
    +    queryConfig: StreamQueryConfig)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    +          with Compiler[FlatJoinFunction[Row, Row, Row]] {
    +
    +  // state to hold left stream element
    +  private var leftState: MapState[Row, JTuple2[Int, Long]] = _
    +  // state to hold right stream element
    +  private var rightState: MapState[Row, JTuple2[Int, Long]] = _
    +  private var cRowWrapper: CRowWrappingMultiOuputCollector = _
    +
    +  private val minRetentionTime: Long = 
queryConfig.getMinIdleStateRetentionTime
    +  private val maxRetentionTime: Long = 
queryConfig.getMaxIdleStateRetentionTime
    +  private val stateCleaningEnabled: Boolean = minRetentionTime > 1
    +
    +  // state to record last timer of left stream, 0 means no timer
    +  private var leftTimer: ValueState[Long] = _
    +  // state to record last timer of right stream, 0 means no timer
    +  private var rightTimer: ValueState[Long] = _
    +
    +  // other condition function
    +  private var joinFunction: FlatJoinFunction[Row, Row, Row] = _
    +
    +  val LOG = LoggerFactory.getLogger(this.getClass)
    +
    +  override def open(parameters: Configuration): Unit = {
    +    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()
    +
    +    // initialize left and right state, the first element of tuple2 
indicates how many rows of
    +    // this row, while the second element represents the expired time of 
this row.
    +    val tupleTypeInfo = new TupleTypeInfo[JTuple2[Int, Long]](Types.INT, 
Types.LONG)
    +    val leftStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, 
Long]](
    +      "left", leftType, tupleTypeInfo)
    +    val rightStateDescriptor = new MapStateDescriptor[Row, JTuple2[Int, 
Long]](
    +      "right", rightType, tupleTypeInfo)
    +    leftState = getRuntimeContext.getMapState(leftStateDescriptor)
    +    rightState = getRuntimeContext.getMapState(rightStateDescriptor)
    +
    +    // initialize timer state
    +    val valueStateDescriptor1 = new 
ValueStateDescriptor[Long]("timervaluestate1", classOf[Long])
    +    leftTimer = getRuntimeContext.getState(valueStateDescriptor1)
    +    val valueStateDescriptor2 = new 
ValueStateDescriptor[Long]("timervaluestate2", classOf[Long])
    +    rightTimer = getRuntimeContext.getState(valueStateDescriptor2)
    +
    +    cRowWrapper = new CRowWrappingMultiOuputCollector()
    +  }
    +
    +  /**
    +    * 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, leftTimer, leftState, rightState, 
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, rightTimer, rightState, leftState, 
false)
    +  }
    +
    +
    +  /**
    +    * Called when a processing timer trigger.
    +    * Expire left/right records which are expired in left and right state.
    +    *
    +    * @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 (stateCleaningEnabled && leftTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        leftState,
    +        leftTimer,
    +        ctx
    +      )
    +    }
    +
    +    if (stateCleaningEnabled && rightTimer.value == timestamp) {
    +      expireOutTimeRow(
    +        timestamp,
    +        rightState,
    +        rightTimer,
    +        ctx
    +      )
    +    }
    +  }
    +
    +
    +  def getNewExpiredTime(
    +      curProcessTime: Long,
    +      oldExpiredTime: Long): Long = {
    +
    +    if (stateCleaningEnabled && curProcessTime + minRetentionTime > 
oldExpiredTime) {
    +      curProcessTime + maxRetentionTime
    +    } else {
    +      oldExpiredTime
    +    }
    +  }
    +
    +  /**
    +    * Puts or Retract an element from the input stream into state and 
search the other state to
    +    * output records meet the condition. Records will be expired in state 
if state retention time
    +    * has been specified.
    +    */
    +  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 = {
    +
    +    cRowWrapper.setCollector(out)
    +    cRowWrapper.setChange(value.change)
    +
    +    val curProcessTime = ctx.timerService.currentProcessingTime
    +    val oldCntAndExpiredTime = currentSideState.get(value.row)
    --- End diff --
    
    This only works given that https://issues.apache.org/jira/browse/FLINK-8139 
is fixed.


---

Reply via email to