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

Jark Wu commented on FLINK-16265:
---------------------------------

I think this is also introduced by FLINK-15912. I reproduced the problem using 
the following yaml:


{code:java}
tables:
  - name: lineitem
    type: source-table
    update-mode: append
    connector:
      type: filesystem
      path: "$TABLE_DIR/lineitem.csv"
    format:
      type: csv
      fields:
        - name: l_linestatus
          type: VARCHAR
        - name: l_shipdate
          type: DATE
        - name: l_commitdate
          type: TIMESTAMP
      field-delimiter: "|"
      line-delimiter: "\n"
      comment-prefix: "--"
    schema:
      - name: l_linestatus
        type: VARCHAR
      - name: l_shipdate
        type: DATE
      - name: l_commitdate
        type: TIMESTAMP
{code}

The root cause is we will convert the properties into CatalogTableImpl and then 
convert into properties again. The schema type properties will use new type 
systems then which is not equal to the legacy types. 



> TPC-H end-to-end test (Blink Planner): Encodings that differ from the schema 
> are not supported yet for CsvTableSources.
> -----------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-16265
>                 URL: https://issues.apache.org/jira/browse/FLINK-16265
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.11.0
>            Reporter: Robert Metzger
>            Priority: Major
>              Labels: test-stability
>
> https://travis-ci.org/apache/flink/jobs/654409371
> {code}
> Dispatcher REST endpoint is up.
> Running query #1...
> Exception in thread "main" org.apache.flink.table.client.SqlClientException: 
> Unexpected exception. This is a bug. Please consider filing an issue.
>       at org.apache.flink.table.client.SqlClient.main(SqlClient.java:190)
> Caused by: org.apache.flink.table.client.gateway.SqlExecutionException: Could 
> not create execution context.
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext$Builder.build(ExecutionContext.java:779)
>       at 
> org.apache.flink.table.client.gateway.local.LocalExecutor.openSession(LocalExecutor.java:228)
>       at org.apache.flink.table.client.SqlClient.start(SqlClient.java:98)
>       at org.apache.flink.table.client.SqlClient.main(SqlClient.java:178)
> Caused by: org.apache.flink.table.api.TableException: Encodings that differ 
> from the schema are not supported yet for CsvTableSources.
>       at 
> org.apache.flink.table.sources.CsvTableSourceFactoryBase.createTableSource(CsvTableSourceFactoryBase.java:127)
>       at 
> org.apache.flink.table.sources.CsvAppendTableSourceFactory.createStreamTableSource(CsvAppendTableSourceFactory.java:46)
>       at 
> org.apache.flink.table.factories.StreamTableSourceFactory.createTableSource(StreamTableSourceFactory.java:55)
>       at 
> org.apache.flink.table.factories.TableSourceFactory.createTableSource(TableSourceFactory.java:63)
>       at 
> org.apache.flink.table.factories.TableSourceFactory.createTableSource(TableSourceFactory.java:74)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.createTableSource(ExecutionContext.java:384)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.lambda$initializeCatalogs$6(ExecutionContext.java:585)
>       at java.util.LinkedHashMap.forEach(LinkedHashMap.java:684)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.initializeCatalogs(ExecutionContext.java:583)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.initializeTableEnvironment(ExecutionContext.java:520)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.<init>(ExecutionContext.java:165)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext.<init>(ExecutionContext.java:122)
>       at 
> org.apache.flink.table.client.gateway.local.ExecutionContext$Builder.build(ExecutionContext.java:768)
>       ... 3 more
> [FAIL] Test script contains errors.
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to