[ https://issues.apache.org/jira/browse/FLINK-9715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16633997#comment-16633997 ]
ASF GitHub Bot commented on FLINK-9715: --------------------------------------- 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_r221569725 ########## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TemporalProcessTimeJoin.scala ########## @@ -20,24 +20,24 @@ package org.apache.flink.table.runtime.join import org.apache.flink.api.common.functions.FlatJoinFunction import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor} import org.apache.flink.api.common.typeinfo.TypeInformation -import org.apache.flink.configuration.Configuration -import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.streaming.api.operators.{AbstractStreamOperator, TimestampedCollector, TwoInputStreamOperator} +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.Collector -class TemporalJoin( +class TemporalProcessTimeJoin( leftType: TypeInformation[Row], rightType: TypeInformation[Row], genJoinFuncName: String, genJoinFuncCode: String, queryConfig: StreamQueryConfig) - extends CoProcessFunction[CRow, CRow, CRow] + extends AbstractStreamOperator[CRow] Review comment: We should have a clear separation between Flink functions and operators. All operator extensions should end up in `org.apache.flink.table.runtime.operators` and should only contain minimal changes that we could contribute back to `flink-core`. I suggest to undo the changes in this class. ---------------------------------------------------------------- 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)