[ 
https://issues.apache.org/jira/browse/FLINK-5662?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15863046#comment-15863046
 ] 

ASF GitHub Bot commented on FLINK-5662:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3277#discussion_r100708670
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/CommonScan.scala
 ---
    @@ -0,0 +1,98 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.table.plan.nodes
    +
    +import org.apache.flink.api.common.functions.MapFunction
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.typeutils.RowTypeInfo
    +import org.apache.flink.table.api.TableConfig
    +import org.apache.flink.table.codegen.CodeGenerator
    +import org.apache.flink.table.runtime.MapRunner
    +import org.apache.flink.types.Row
    +
    +/**
    +  * Common class for batch and stream scans.
    +  */
    +trait CommonScan {
    +
    +  /**
    +    * We check if the input type is exactly the same as the internal row 
type.
    +    * A conversion is necessary if types differ or object have to be 
unboxed
    +    * (i.e. Date, Time, Timestamp need to be converted into their 
primitive equivalents).
    +    */
    +  private[flink] def needsConversion(
    +      externalTypeInfo: TypeInformation[Any],
    +      internalTypeInfo: TypeInformation[Row])
    +    : Boolean = {
    +
    +    if (externalTypeInfo == internalTypeInfo) {
    +      val rowTypeInfo = externalTypeInfo.asInstanceOf[RowTypeInfo]
    +      var containsBoxedTypes = false
    +      // TODO enable these lines for FLINK-5429
    --- End diff --
    
    We should either address FLINK-5429 in this PR or remove the related code.


> 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)

Reply via email to