lincoln-lil commented on code in PR #20899:
URL: https://github.com/apache/flink/pull/20899#discussion_r981175636
##########
flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/OperatorCodeGenerator.scala:
##########
@@ -109,10 +109,16 @@ object OperatorCodeGenerator extends Logging {
$endInput
+ @Override
+ public void finish() throws Exception {
+ ${ctx.reuseFinishCode()}
+ super.finish();
Review Comment:
similar to the close sequence, it is relatively more safer (not always
strictly) to handle the actions of the child class first and call the parent
public logic last. Btw, I changed the current close call at first but reverted
it for focusing on this pr itself.
##########
flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml:
##########
@@ -25,7 +25,7 @@ LogicalJoin(condition=[=($3, $1)], joinType=[inner])
: +- LogicalCorrelate(correlation=[$cor0], joinType=[inner],
requiredColumns=[{}])
: :- LogicalProject(o_rowtime=[AS($0, _UTF-16LE'o_rowtime')],
o_comment=[AS($1, _UTF-16LE'o_comment')], o_amount=[AS($2,
_UTF-16LE'o_amount')], o_currency=[AS($3, _UTF-16LE'o_currency')],
o_secondary_key=[AS($4, _UTF-16LE'o_secondary_key')])
: : +- LogicalTableScan(table=[[default_catalog, default_database,
Orders]])
-: +-
LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$bbf912e58a3fb2d083552961c0f87dbe*($0)],
rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647)
comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)])
Review Comment:
I did try to add the uid, but it doesn't work in the current
TemporalTableFunction implementation, it was wrapped into a BridgingSqlFunction
(a subclass of calcite's SqlFunction which is not serializable) during the
conversion, so give up the change for now.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]