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

    https://github.com/apache/flink/pull/4625#discussion_r139727416
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/ProcTimeBoundedStreamInnerJoin.scala
 ---
    @@ -0,0 +1,74 @@
    +/*
    + * 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.typeinfo.TypeInformation
    +import org.apache.flink.streaming.api.functions.co.CoProcessFunction
    +import org.apache.flink.table.runtime.types.CRow
    +import org.apache.flink.types.Row
    +
    +/**
    +  * The function to execute processing time bounded stream inner-join.
    +  */
    +class ProcTimeBoundedStreamInnerJoin(
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    allowedLateness: Long,
    +    leftType: TypeInformation[Row],
    +    rightType: TypeInformation[Row],
    +    genJoinFuncName: String,
    +    genJoinFuncCode: String)
    +    extends TimeBoundedStreamInnerJoin(
    +      leftLowerBound,
    +      leftUpperBound,
    +      allowedLateness,
    +      leftType,
    +      rightType,
    +      genJoinFuncName,
    +      genJoinFuncCode,
    +      leftTimeIdx = -1,
    +      rightTimeIdx = -1,
    +      JoinTimeIndicator.PROCTIME) {
    +
    +  override def checkRowOutOfDate(timeForRow: Long, watermark: Long) = false
    +
    +  override def updateOperatorTime(ctx: CoProcessFunction[CRow, CRow, 
CRow]#Context): Unit = {
    +    rightOperatorTime = ctx.timerService().currentProcessingTime()
    +    leftOperatorTime = ctx.timerService().currentProcessingTime()
    +  }
    +
    +  override def getTimeForLeftStream(
    +      context: CoProcessFunction[CRow, CRow, CRow]#Context,
    +      row: CRow): Long = {
    +    context.timerService().currentProcessingTime()
    --- End diff --
    
    To be honest, I would not put too much effort into the processing time 
case, especially not if it affects the performance of event-time processing. 
Processing time is non-deterministic anyway. The reason I brought this up is 
because I wasn't sure of the side effects if the the row proctime > operator 
time. If this is not an issue, we can keep it like this. 
    
    Otherwise, the easiest solution would be to just add a comment to the 
invocations of `updateOperatorTime` that this call must be the first call in 
all processing methods (`processElement()`, `onTimer()`). Since this is just 
internal API, this should be fine.


---

Reply via email to