[ 
https://issues.apache.org/jira/browse/FLINK-9715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16641007#comment-16641007
 ] 

ASF GitHub Bot commented on FLINK-9715:
---------------------------------------

hequn8128 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_r223206551
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala
 ##########
 @@ -60,32 +61,28 @@ class TemporalJoin(
     val rightStateDescriptor = new ValueStateDescriptor[Row]("right", 
rightType)
     rightState = getRuntimeContext.getState(rightStateDescriptor)
 
+    collector = new TimestampedCollector[CRow](output)
     cRowWrapper = new CRowWrappingCollector()
+    cRowWrapper.out = collector
   }
 
-  override def processElement1(
-      value: CRow,
-      ctx: CoProcessFunction[CRow, CRow, CRow]#Context,
-      out: Collector[CRow]): Unit = {
+  override def processElement1(element: StreamRecord[CRow]): Unit = {
 
     if (rightState.value() == null) {
       return
     }
 
-    cRowWrapper.out = out
-    cRowWrapper.setChange(value.change)
+    cRowWrapper.setChange(element.getValue.change)
+    collector.setTimestamp(element)
 
 Review comment:
   Why call this function? The value of StreamRecord timestamp has already been 
erased. 

----------------------------------------------------------------
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:
us...@infra.apache.org


> 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)

Reply via email to