[ https://issues.apache.org/jira/browse/FLINK-10638?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16668911#comment-16668911 ]
ASF GitHub Bot commented on FLINK-10638: ---------------------------------------- pnowojski closed pull request #6893: [FLINK-10638] [table] Fix table scan resolution for temporal join queries URL: https://github.com/apache/flink/pull/6893 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala index 65191202ad8..6d6a17d9e8f 100644 --- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala +++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala @@ -804,9 +804,9 @@ abstract class StreamTableEnvironment( */ private[flink] def optimize(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = { val convSubQueryPlan = optimizeConvertSubQueries(relNode) - val temporalTableJoinPlan = optimizeConvertToTemporalJoin(convSubQueryPlan) - val fullNode = optimizeConvertTableReferences(temporalTableJoinPlan) - val decorPlan = RelDecorrelator.decorrelateQuery(fullNode) + val fullNode = optimizeConvertTableReferences(convSubQueryPlan) + val temporalTableJoinPlan = optimizeConvertToTemporalJoin(fullNode) + val decorPlan = RelDecorrelator.decorrelateQuery(temporalTableJoinPlan) val planWithMaterializedTimeAttributes = RelTimeIndicatorConverter.convert(decorPlan, getRelBuilder.getRexBuilder) val normalizedPlan = optimizeNormalizeLogicalPlan(planWithMaterializedTimeAttributes) diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala index df0f01be0d5..4b327a9fe59 100644 --- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala +++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/sql/TemporalJoinITCase.scala @@ -141,12 +141,12 @@ class TemporalJoinITCase extends StreamingWithStateTestBase { .toTable(tEnv, 'currency, 'rate, 'rowtime.rowtime) tEnv.registerTable("Orders", orders) - tEnv.registerTable("RatesHistory", ratesHistory) tEnv.registerFunction( "Rates", ratesHistory.createTemporalTableFunction('rowtime, 'currency)) + tEnv.registerTable("TemporalJoinResult", tEnv.sqlQuery(sqlQuery)) - val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row] + val result = tEnv.scan("TemporalJoinResult").toAppendStream[Row] result.addSink(new StreamITCase.StringSink[Row]) env.execute() ---------------------------------------------------------------- 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 > Invalid table scan resolution for temporal join queries > ------------------------------------------------------- > > Key: FLINK-10638 > URL: https://issues.apache.org/jira/browse/FLINK-10638 > Project: Flink > Issue Type: Bug > Components: Table API & SQL > Reporter: Timo Walther > Assignee: Piotr Nowojski > Priority: Major > Labels: pull-request-available > > Registered tables that contain a temporal join are not properly resolved when > performing a table scan. > A planning error occurs when registering a table with a temporal join and > reading from it again. > {code} > LogicalProject(amount=[*($0, $4)]) > LogicalFilter(condition=[=($3, $1)]) > LogicalCorrelate(correlation=[$cor0], joinType=[inner], > requiredColumns=[{2}]) > LogicalTableScan(table=[[_DataStreamTable_0]]) > > LogicalTableFunctionScan(invocation=[Rates(CAST($cor0.rowtime):TIMESTAMP(3) > NOT NULL)], rowType=[RecordType(VARCHAR(65536) currency, BIGINT rate, TIME > ATTRIBUTE(ROWTIME) rowtime)], elementType=[class [Ljava.lang.Object;]) > {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)