[
https://issues.apache.org/jira/browse/FLINK-6233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16171021#comment-16171021
]
ASF GitHub Bot commented on FLINK-6233:
---------------------------------------
Github user xccui commented on a diff in the pull request:
https://github.com/apache/flink/pull/4625#discussion_r139585456
--- 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 --
Yes, you are right. To keep them identical, we should return the
`leftOperatorTime` here. However, this makes `updateOperatorTime` and
`getTimeForLeftStream` coupled, i.e., `updateOperatorTime` must be invoked
before `getTimeForLeftStream`. Can we bear this?
I've got an idea about the processing time. How about temporarily caching
the machine time for the same `StreamRecord` instead of invoking the
`System.currentTimeMillis()` each time?
> Support rowtime inner equi-join between two streams in the SQL API
> ------------------------------------------------------------------
>
> Key: FLINK-6233
> URL: https://issues.apache.org/jira/browse/FLINK-6233
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: hongyuhong
> Assignee: Xingcan Cui
>
> The goal of this issue is to add support for inner equi-join on proc time
> streams to the SQL interface.
> Queries similar to the following should be supported:
> {code}
> SELECT o.rowtime , o.productId, o.orderId, s.rowtime AS shipTime
> FROM Orders AS o
> JOIN Shipments AS s
> ON o.orderId = s.orderId
> AND o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL '1' HOUR;
> {code}
> The following restrictions should initially apply:
> * The join hint only support inner join
> * The ON clause should include equi-join condition
> * The time-condition {{o.rowtime BETWEEN s.rowtime AND s.rowtime + INTERVAL
> '1' HOUR}} only can use rowtime that is a system attribute, the time
> condition only support bounded time range like {{o.rowtime BETWEEN s.rowtime
> - INTERVAL '1' HOUR AND s.rowtime + INTERVAL '1' HOUR}}, not support
> unbounded like {{o.rowtime < s.rowtime}} , and should include both two
> stream's rowtime attribute, {{o.rowtime between rowtime () and rowtime () +
> 1}} should also not be supported.
> An row-time streams join will not be able to handle late data, because this
> would mean in insert a row into a sorted order shift all other computations.
> This would be too expensive to maintain. Therefore, we will throw an error if
> a user tries to use an row-time stream join with late data handling.
> This issue includes:
> * Design of the DataStream operator to deal with stream join
> * Translation from Calcite's RelNode representation (LogicalJoin).
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)