Re: How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Danny Chan
You can not do that in Flink yet, Flink partition column must be mapped to columns from the table schema which you can select from. The syntax is a little different from Hive’s => create table table_name (   idint,   dtDontQuery   string,   name  string )

Re: How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Dongwon Kim
Thanks Jark for the update. However, getting back to the original question, can I use a nested column directly for CREATE TABLE PARTITIONED BY like below without declaring an additional column? CREATE TABLE output > PARTITIONED BY (`location.transId`) > WITH ( > 'connector' = 'filesystem', >

Re: How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Jark Wu
Hi Dongwon, I think this is a bug in the Filesystem connector which doesn't exclude the computed columns when building the TableSource. I created an issue [1] to track this problem. Best, Jark [1]: https://issues.apache.org/jira/browse/FLINK-18665 On Tue, 21 Jul 2020 at 17:31, Dongwon Kim

Re: How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Dongwon Kim
Hi Danny, Which version did you use I use Flink 1.11.0. > what SQL context throws the error ? I think the declaration itself is not a problem. The exception occurs when I tried to execute the following which I didn't show you in the previous email: > tEnv.sqlQuery("SELECT type, location

Re: How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Danny Chan
Hi, I execute the sql below """ |create table navi ( | a STRING, | location ROW |) with ( | 'connector' = 'filesystem', | 'path' = 'east-out', | 'format' = 'json' |) |""".stripMargin tableEnv.executeSql(sql0) val sql = """ |CREATE TABLE output ( |

How to use a nested column for CREATE TABLE PARTITIONED BY

2020-07-21 Thread Dongwon Kim
Hi, I want to create subdirectories named after values of a nested column, location.transId. This is my first attempt: > CREATE TABLE output > PARTITIONED BY (`location.transId`) > WITH ( > 'connector' = 'filesystem', > 'path' = 'east-out', > 'format' = 'json' > ) LIKE navi (EXCLUDING