You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/10/01 13:32:24 UTC

[GitHub] twalthr commented on a change in pull request #6776: [FLINK-9715][table] Support temporal join with event time

twalthr commented on a change in pull request #6776: [FLINK-9715][table] Support temporal join with event time
URL: https://github.com/apache/flink/pull/6776#discussion_r221583273
 
 

 ##########
 File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
 ##########
 @@ -0,0 +1,307 @@
+/*
+ * 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.lang.{Long => JLong}
+import java.util
+import java.util.Comparator
+
+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.runtime.state.{VoidNamespace, VoidNamespaceSerializer}
+import org.apache.flink.streaming.api.SimpleTimerService
+import org.apache.flink.streaming.api.operators._
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
+import org.apache.flink.table.api.{StreamQueryConfig, TableException}
+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.typeutils.TypeCheckUtils._
+import org.apache.flink.table.util.Logging
+import org.apache.flink.types.Row
+
+import scala.collection.JavaConversions._
+
+/**
+  * This operator works by keeping on the state collection of probe and build records to process
+  * on next watermark. The idea is that between watermarks we are collecting those elements
+  * and once we are sure that there will be no updates we emit the correct result and clean up the
+  * state.
+  *
+  * Cleaning up the state drops all of the "old" values from the probe side, where "old" is defined
+  * as older then the current watermark. Build side is also cleaned up in the similar fashion,
+  * however we always keep at least one record - the latest one - even if it's past the last
+  * watermark.
+  *
+  * One more trick is how the emitting results and cleaning up is triggered. It is achieved
+  * by registering timers for the keys. We could register a timer for every probe and build
+  * side element's event time (when watermark exceeds this timer, that's when we are emitting and/or
+  * cleaning up the state). However this would cause huge number of registered timers. For example
+  * with following evenTimes of probe records accumulated: {1, 2, 5, 8, 9}, if we
+  * had received Watermark(10), it would trigger 5 separate timers for the same key. To avoid that
+  * we always keep only one single registered timer for any given key, registered for the minimal
+  * value. Upon triggering it, we process all records with event times older then currentWatermark.
+  */
+class TemporalRowtimeJoin(
+    leftType: TypeInformation[Row],
+    rightType: TypeInformation[Row],
+    genJoinFuncName: String,
+    genJoinFuncCode: String,
+    queryConfig: StreamQueryConfig,
+    leftTimeAttribute: Int,
+    rightTimeAttribute: Int)
+  extends AbstractStreamOperator[CRow]
+  with TwoInputStreamOperator[CRow, CRow, CRow]
+  with Triggerable[Any, VoidNamespace]
+  with Compiler[FlatJoinFunction[Row, Row, Row]]
+  with Logging {
+
+  validateEqualsHashCode("join", leftType)
+  validateEqualsHashCode("join", rightType)
+
+  private val netLeftIndexStateName = "next-index"
+  private val leftStateName = "left"
+  private val rightStateName = "right"
+  private val registeredTimerStateName = "timer"
+  private val probteTimersStateName = "probe-timers"
 
 Review comment:
   btw I find the mixture of left/right, probe/build confusing. Maybe call this `rightTimersStateName` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services