[
https://issues.apache.org/jira/browse/FLINK-5662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15859284#comment-15859284
]
ASF GitHub Bot commented on FLINK-5662:
---------------------------------------
Github user KurtYoung commented on a diff in the pull request:
https://github.com/apache/flink/pull/3277#discussion_r100270646
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
---
@@ -285,28 +286,37 @@ abstract class BatchTableEnvironment(
* @return The [[DataSet]] that corresponds to the translated [[Table]].
*/
protected def translate[A](table: Table)(implicit tpe:
TypeInformation[A]): DataSet[A] = {
- val dataSetPlan = optimize(table.getRelNode)
- translate(dataSetPlan)
+ val relNode = table.getRelNode
+ val dataSetPlan = optimize(relNode)
+ translate(dataSetPlan, relNode.getRowType)
}
/**
- * Translates a logical [[RelNode]] into a [[DataSet]].
+ * Translates a logical [[RelNode]] into a [[DataSet]]. Converts to
target type if necessary.
*
* @param logicalPlan The root node of the relational expression tree.
* @param tpe The [[TypeInformation]] of the resulting
[[DataSet]].
* @tparam A The type of the resulting [[DataSet]].
* @return The [[DataSet]] that corresponds to the translated [[Table]].
*/
- protected def translate[A](logicalPlan: RelNode)(implicit tpe:
TypeInformation[A]): DataSet[A] = {
+ protected def translate[A](
+ logicalPlan: RelNode,
+ logicalType: RelDataType)
--- End diff --
And if it is indeed needed, we should update method comment as well
> Alias in front of output fails
> ------------------------------
>
> Key: FLINK-5662
> URL: https://issues.apache.org/jira/browse/FLINK-5662
> Project: Flink
> Issue Type: Bug
> Components: Table API & SQL
> Affects Versions: 1.2.0
> Reporter: Timo Walther
> Assignee: Timo Walther
>
> Neither of the following snippets works:
> {code}
> public static void main(String[] args) throws Exception {
> final ExecutionEnvironment env =
> ExecutionEnvironment.getExecutionEnvironment();
> DataSet<String> text = ...
> BatchTableEnvironment tEnv = TableEnvironment.getTableEnvironment(env);
> // Table t = tEnv.fromDataSet(text, "text").select("text AS line");
> // Table t = tEnv.fromDataSet(text, "text").as("line");
> Table t = tEnv.fromDataSet(text, "text").select("text AS
> line").select("line AS line");
> tEnv.toDataSet(t, MyPojo.class).print();
> }
> public static class MyPojo {
> public String line;
> }
> {code}
> {code}
> Exception in thread "main" org.apache.flink.table.api.TableException: POJO
> does not define field name: text
> at
> org.apache.flink.table.typeutils.TypeConverter$$anonfun$2.apply(TypeConverter.scala:85)
> at
> org.apache.flink.table.typeutils.TypeConverter$$anonfun$2.apply(TypeConverter.scala:81)
> at scala.collection.immutable.List.foreach(List.scala:318)
> at
> org.apache.flink.table.typeutils.TypeConverter$.determineReturnType(TypeConverter.scala:81)
> at
> org.apache.flink.table.plan.nodes.dataset.BatchScan.convertToExpectedType(BatchScan.scala:69)
> at
> org.apache.flink.table.plan.nodes.dataset.DataSetScan.translateToPlan(DataSetScan.scala:61)
> at
> org.apache.flink.table.api.BatchTableEnvironment.translate(BatchTableEnvironment.scala:305)
> at
> org.apache.flink.table.api.BatchTableEnvironment.translate(BatchTableEnvironment.scala:289)
> at
> org.apache.flink.table.api.java.BatchTableEnvironment.toDataSet(BatchTableEnvironment.scala:146)
> at groupId.WordCount.main(WordCount.java:67)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:498)
> at com.intellij.rt.execution.application.AppMain.main(AppMain.java:147)
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)