[
https://issues.apache.org/jira/browse/FLINK-9715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651584#comment-16651584
]
ASF GitHub Bot commented on FLINK-9715:
---------------------------------------
kl0u 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_r225511761
##########
File path:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalRowtimeJoin.scala
##########
@@ -0,0 +1,339 @@
+/*
+ * 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
+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 org.apache.flink.util.Preconditions.checkState
+
+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 or equal to
+ * 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 NEXT_LEFT_INDEX_STATE_NAME = "next-index"
+ private val LEFT_STATE_NAME = "left"
+ private val RIGHT_STATE_NAME = "right"
+ private val REGISTERED_TIMER_STATE_NAME = "timer"
+ private val TIMERS_STATE_NAME = "timers"
+
+ private val rightRowtimeComparator = new
RowtimeComparator(rightTimeAttribute)
+
+ /**
+ * Incremental index generator for `leftState`'s keys.
+ */
+ private var nextLeftIndex: ValueState[JLong] = _
+
+ /**
+ * Mapping from artificial row index (generated by `nextLeftIndex`) into
the left side `Row`.
+ * We can not use List to accumulate Rows, because we need efficient
deletes of the oldest rows.
+ *
+ * TODO: this could be OrderedMultiMap[Jlong, Row] indexed by row's
timestamp, to avoid
+ * full map traversals (if we have lots of rows on the state that exceed
`currentWatermark`).
+ */
Review comment:
As far as the number of lists is concerned, this is not an issue, as we can
have a single list that we just `clear()` and repopulate for each element. This
I think also partially answers the "significantly slow down for the common
usecase" argument (no significant GC oberhead), for which actually I disagree
in the first place, as, I would assume, that a RocksDB access is far more
expensive than creating a list. In addition, I am not so sure what is the
"common" usecase. Under the label "common", I would only put RocksDB as state
backend, as this is what we normally recommend for production usecases. What
are the other assumptions that we should consider as common? If it is that we
have a single element per timestamp, then, again, a single list that we clear
and repopulate incurs a minimal overhead.
One final argument is that having a timestamp as a key would improve code
readability. The reason why I focused on this in the first place, was that I
could not understand the reason why we have as key an arbitrary index, which is
an overhead in terms of storage and state accesses.
----------------------------------------------------------------
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:
[email protected]
> Support versioned joins with event time
> ---------------------------------------
>
> Key: FLINK-9715
> URL: https://issues.apache.org/jira/browse/FLINK-9715
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Affects Versions: 1.5.0
> Reporter: Piotr Nowojski
> Assignee: Piotr Nowojski
> Priority: Major
> Labels: pull-request-available
>
> Queries like:
> {code:java}
> SELECT
> o.amount * r.rate
> FROM
> Orders AS o,
> LATERAL TABLE (Rates(o.rowtime)) AS r
> WHERE o.currency = r.currency{code}
> should work with event time
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)