Hequn Cheng created FLINK-12779: ----------------------------------- Summary: Avoid field conflicts when generate field names for non-composite Typeinformation Key: FLINK-12779 URL: https://issues.apache.org/jira/browse/FLINK-12779 Project: Flink Issue Type: Improvement Components: Table SQL / API Reporter: Hequn Cheng Assignee: Hequn Cheng
We use {{FieldInfoUtils.getFieldNames(resultType)}} to get the relative field names of the resultType. There are no problem for composite types. For non-composite types, we always set the field name to `f0`. But the `f0` may conflict with the predefined field names. To make it more robust, we should generate a field name with no conflicts. For example, we can use `f0_0` as the field name if `f0` has been used. This is also consistent with the behavior of SQL. The following test can reproduce the problem. {code:java} @Test def testUserDefinedTableFunctionWithParameter(): Unit = { val tableFunc1 = new RichTableFunc1 StreamITCase.testResults = mutable.MutableList() val result = StreamTestData.getSmall3TupleDataStream(env) .toTable(tEnv, 'f0, 'f1, 'f2) .joinLateral(tableFunc1('f2)) val results = result.toAppendStream[Row] results.addSink(new StreamITCase.StringSink[Row]) env.execute() val expected = mutable.MutableList("3,Hello", "3,world") assertEquals(expected.sorted, StreamITCase.testResults.sorted) } {code} Exception {code:java} org.apache.flink.table.api.ValidationException: join relations with ambiguous names: [f0] at org.apache.flink.table.operations.JoinOperationFactory.validateNamesAmbiguity(JoinOperationFactory.java:115) at org.apache.flink.table.operations.JoinOperationFactory.create(JoinOperationFactory.java:78) at org.apache.flink.table.operations.OperationTreeBuilder.join(OperationTreeBuilder.scala:358) at org.apache.flink.table.operations.OperationTreeBuilder.joinLateral(OperationTreeBuilder.scala:373) at org.apache.flink.table.api.TableImpl.joinLateralInternal(tableImpl.scala:256) at org.apache.flink.table.api.TableImpl.joinLateral(tableImpl.scala:214) {code} -- This message was sent by Atlassian JIRA (v7.6.3#76005)