CREATE TABLE LIKE clause from different catalog or database

classic Classic list List threaded Threaded
6 messages Options
Reply | Threaded
Open this post in threaded view
|

CREATE TABLE LIKE clause from different catalog or database

Dongwon Kim-2
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon
Reply | Threaded
Open this post in threaded view
|

Re: CREATE TABLE LIKE clause from different catalog or database

Danny Chan-2
Hi Dongwon ~

Table from different catalog/db is supported, you need to specify the full path of the source table:

CREATE TABLE Orders_with_watermark (
    ...
) WITH (
    ...
)
LIKE my_catalog.my_db.Orders;

Dongwon Kim <[hidden email]> 于2020年11月11日周三 下午2:53写道:
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon
Reply | Threaded
Open this post in threaded view
|

Re: CREATE TABLE LIKE clause from different catalog or database

Dongwon Kim-2
Hi Danny~
Sorry for late reply,

Let's take a look at a running example:
EnvironmentSettings settings = EnvironmentSettings.newInstance()
  .inBatchMode()
  .build();

TableEnvironment tEnv = TableEnvironment.create(settings);

HiveCatalog hiveCatalog = new HiveCatalog("hive",null, args[1]);
tEnv.registerCatalog("hive", hiveCatalog);

GenericInMemoryCatalog inmemCatalog = new GenericInMemoryCatalog("inmem");
tEnv.registerCatalog("inmem", inmemCatalog);
tEnv.useCatalog("inmem");

TableResult result = tEnv.executeSql(
  "CREATE TABLE copied LIKE hive.navi.gps"
);

I've got the following log messages:
00:50:22,157 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Setting hive conf dir as /Users/eastcirclek/hive-conf
00:50:22,503 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Created HiveCatalog 'hive'
00:50:22,515 INFO  hive.metastore                                               [] - Trying to connect to metastore with URI thrift://...:9083
00:50:22,678 INFO  hive.metastore                                               [] - Connected to metastore.
00:50:22,678 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Connected to Hive metastore
00:50:22,799 INFO  org.apache.flink.table.catalog.CatalogManager                [] - Set the current default catalog as [inmem] and the current default database as [default].
Exception in thread "main" org.apache.flink.table.api.ValidationException: Source table '`inmem`.`default`.`hive.navi.gps`' of the LIKE clause not found in the catalog, at line 1, column 26
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lambda$lookupLikeSourceTable$1(SqlCreateTableConverter.java:147)
at java.util.Optional.orElseThrow(Optional.java:290)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lookupLikeSourceTable(SqlCreateTableConverter.java:147)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.createCatalogTable(SqlCreateTableConverter.java:96)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.convertCreateTable(SqlCreateTableConverter.java:76)
at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:190)
at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:78)
at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:684)
at Test.main(Test.java:53)

It seems like hive.navi.gps is recognized as a table name as a whole. 
I currently declare such a table by specifying all fields without the LIKE clause.

Do I miss something?

FYI, I'm working with Flink-1.11.2.

Thank you~

Best,

Dongwon


On Fri, Nov 13, 2020 at 5:19 PM Danny Chan <[hidden email]> wrote:
Hi Dongwon ~

Table from different catalog/db is supported, you need to specify the full path of the source table:

CREATE TABLE Orders_with_watermark (
    ...
) WITH (
    ...
)
LIKE my_catalog.my_db.Orders;

Dongwon Kim <[hidden email]> 于2020年11月11日周三 下午2:53写道:
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon
Reply | Threaded
Open this post in threaded view
|

Re: CREATE TABLE LIKE clause from different catalog or database

Dongwon Kim-2
In reply to this post by Dongwon Kim-2
Hi Ingo, 

Thank you for letting me know! I didn’t know that’s already discussed. 

Best,

Dongwon

2020. 11. 17. 오전 1:12, Ingo Bürk <[hidden email]> 작성:


Hi,

I ran into the same issue today. This is fixed in 1.11.3, the corresponding bug was FLINK-19281.

A workaround is to switch the current catalog and database temporarily to hive.navi and then not qualify the table name in the LIKE clause.


Regards
Ingo


On Mon, Nov 16, 2020, 17:04 Dongwon Kim <[hidden email]> wrote:
Hi Danny~
Sorry for late reply,

Let's take a look at a running example:
EnvironmentSettings settings = EnvironmentSettings.newInstance()
  .inBatchMode()
  .build();

TableEnvironment tEnv = TableEnvironment.create(settings);

HiveCatalog hiveCatalog = new HiveCatalog("hive",null, args[1]);
tEnv.registerCatalog("hive", hiveCatalog);

GenericInMemoryCatalog inmemCatalog = new GenericInMemoryCatalog("inmem");
tEnv.registerCatalog("inmem", inmemCatalog);
tEnv.useCatalog("inmem");

TableResult result = tEnv.executeSql(
  "CREATE TABLE copied LIKE hive.navi.gps"
);

I've got the following log messages:
00:50:22,157 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Setting hive conf dir as /Users/eastcirclek/hive-conf
00:50:22,503 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Created HiveCatalog 'hive'
00:50:22,515 INFO  hive.metastore                                               [] - Trying to connect to metastore with URI thrift://...:9083
00:50:22,678 INFO  hive.metastore                                               [] - Connected to metastore.
00:50:22,678 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Connected to Hive metastore
00:50:22,799 INFO  org.apache.flink.table.catalog.CatalogManager                [] - Set the current default catalog as [inmem] and the current default database as [default].
Exception in thread "main" org.apache.flink.table.api.ValidationException: Source table '`inmem`.`default`.`hive.navi.gps`' of the LIKE clause not found in the catalog, at line 1, column 26
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lambda$lookupLikeSourceTable$1(SqlCreateTableConverter.java:147)
at java.util.Optional.orElseThrow(Optional.java:290)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lookupLikeSourceTable(SqlCreateTableConverter.java:147)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.createCatalogTable(SqlCreateTableConverter.java:96)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.convertCreateTable(SqlCreateTableConverter.java:76)
at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:190)
at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:78)
at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:684)
at Test.main(Test.java:53)

It seems like hive.navi.gps is recognized as a table name as a whole. 
I currently declare such a table by specifying all fields without the LIKE clause.

Do I miss something?

FYI, I'm working with Flink-1.11.2.

Thank you~

Best,

Dongwon


On Fri, Nov 13, 2020 at 5:19 PM Danny Chan <[hidden email]> wrote:
Hi Dongwon ~

Table from different catalog/db is supported, you need to specify the full path of the source table:

CREATE TABLE Orders_with_watermark (
    ...
) WITH (
    ...
)
LIKE my_catalog.my_db.Orders;

Dongwon Kim <[hidden email]> 于2020年11月11日周三 下午2:53写道:
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon
Reply | Threaded
Open this post in threaded view
|

Re: CREATE TABLE LIKE clause from different catalog or database

Danny Chan-2
Yes, this is a bug which is fixed recently [1] for release 1.12 and 1.11.3

You can also switch to the source table catalog first before you execute the CREATE TABLE LIKE DDL just like Ingo suggested.


김동원 <[hidden email]> 于2020年11月17日周二 上午12:19写道:
Hi Ingo, 

Thank you for letting me know! I didn’t know that’s already discussed. 

Best,

Dongwon

2020. 11. 17. 오전 1:12, Ingo Bürk <[hidden email]> 작성:


Hi,

I ran into the same issue today. This is fixed in 1.11.3, the corresponding bug was FLINK-19281.

A workaround is to switch the current catalog and database temporarily to hive.navi and then not qualify the table name in the LIKE clause.


Regards
Ingo


On Mon, Nov 16, 2020, 17:04 Dongwon Kim <[hidden email]> wrote:
Hi Danny~
Sorry for late reply,

Let's take a look at a running example:
EnvironmentSettings settings = EnvironmentSettings.newInstance()
  .inBatchMode()
  .build();

TableEnvironment tEnv = TableEnvironment.create(settings);

HiveCatalog hiveCatalog = new HiveCatalog("hive",null, args[1]);
tEnv.registerCatalog("hive", hiveCatalog);

GenericInMemoryCatalog inmemCatalog = new GenericInMemoryCatalog("inmem");
tEnv.registerCatalog("inmem", inmemCatalog);
tEnv.useCatalog("inmem");

TableResult result = tEnv.executeSql(
  "CREATE TABLE copied LIKE hive.navi.gps"
);

I've got the following log messages:
00:50:22,157 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Setting hive conf dir as /Users/eastcirclek/hive-conf
00:50:22,503 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Created HiveCatalog 'hive'
00:50:22,515 INFO  hive.metastore                                               [] - Trying to connect to metastore with URI thrift://...:9083
00:50:22,678 INFO  hive.metastore                                               [] - Connected to metastore.
00:50:22,678 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Connected to Hive metastore
00:50:22,799 INFO  org.apache.flink.table.catalog.CatalogManager                [] - Set the current default catalog as [inmem] and the current default database as [default].
Exception in thread "main" org.apache.flink.table.api.ValidationException: Source table '`inmem`.`default`.`hive.navi.gps`' of the LIKE clause not found in the catalog, at line 1, column 26
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lambda$lookupLikeSourceTable$1(SqlCreateTableConverter.java:147)
at java.util.Optional.orElseThrow(Optional.java:290)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lookupLikeSourceTable(SqlCreateTableConverter.java:147)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.createCatalogTable(SqlCreateTableConverter.java:96)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.convertCreateTable(SqlCreateTableConverter.java:76)
at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:190)
at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:78)
at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:684)
at Test.main(Test.java:53)

It seems like hive.navi.gps is recognized as a table name as a whole. 
I currently declare such a table by specifying all fields without the LIKE clause.

Do I miss something?

FYI, I'm working with Flink-1.11.2.

Thank you~

Best,

Dongwon


On Fri, Nov 13, 2020 at 5:19 PM Danny Chan <[hidden email]> wrote:
Hi Dongwon ~

Table from different catalog/db is supported, you need to specify the full path of the source table:

CREATE TABLE Orders_with_watermark (
    ...
) WITH (
    ...
)
LIKE my_catalog.my_db.Orders;

Dongwon Kim <[hidden email]> 于2020年11月11日周三 下午2:53写道:
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon
Reply | Threaded
Open this post in threaded view
|

Re: CREATE TABLE LIKE clause from different catalog or database

Benchao Li-2
In reply to this post by Dongwon Kim-2
Hi Dongwon,

You are hitting a known bug[1] which is fixed in 1.11.3 and 1.12.0

Another tip, currently, LIKE clause cannot work with Hive table. (General table stored in hive metastore should work)


Dongwon Kim <[hidden email]> 于2020年11月17日周二 上午12:04写道:
Hi Danny~
Sorry for late reply,

Let's take a look at a running example:
EnvironmentSettings settings = EnvironmentSettings.newInstance()
  .inBatchMode()
  .build();

TableEnvironment tEnv = TableEnvironment.create(settings);

HiveCatalog hiveCatalog = new HiveCatalog("hive",null, args[1]);
tEnv.registerCatalog("hive", hiveCatalog);

GenericInMemoryCatalog inmemCatalog = new GenericInMemoryCatalog("inmem");
tEnv.registerCatalog("inmem", inmemCatalog);
tEnv.useCatalog("inmem");

TableResult result = tEnv.executeSql(
  "CREATE TABLE copied LIKE hive.navi.gps"
);

I've got the following log messages:
00:50:22,157 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Setting hive conf dir as /Users/eastcirclek/hive-conf
00:50:22,503 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Created HiveCatalog 'hive'
00:50:22,515 INFO  hive.metastore                                               [] - Trying to connect to metastore with URI thrift://...:9083
00:50:22,678 INFO  hive.metastore                                               [] - Connected to metastore.
00:50:22,678 INFO  org.apache.flink.table.catalog.hive.HiveCatalog              [] - Connected to Hive metastore
00:50:22,799 INFO  org.apache.flink.table.catalog.CatalogManager                [] - Set the current default catalog as [inmem] and the current default database as [default].
Exception in thread "main" org.apache.flink.table.api.ValidationException: Source table '`inmem`.`default`.`hive.navi.gps`' of the LIKE clause not found in the catalog, at line 1, column 26
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lambda$lookupLikeSourceTable$1(SqlCreateTableConverter.java:147)
at java.util.Optional.orElseThrow(Optional.java:290)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.lookupLikeSourceTable(SqlCreateTableConverter.java:147)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.createCatalogTable(SqlCreateTableConverter.java:96)
at org.apache.flink.table.planner.operations.SqlCreateTableConverter.convertCreateTable(SqlCreateTableConverter.java:76)
at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:190)
at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:78)
at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:684)
at Test.main(Test.java:53)

It seems like hive.navi.gps is recognized as a table name as a whole. 
I currently declare such a table by specifying all fields without the LIKE clause.

Do I miss something?

FYI, I'm working with Flink-1.11.2.

Thank you~

Best,

Dongwon


On Fri, Nov 13, 2020 at 5:19 PM Danny Chan <[hidden email]> wrote:
Hi Dongwon ~

Table from different catalog/db is supported, you need to specify the full path of the source table:

CREATE TABLE Orders_with_watermark (
    ...
) WITH (
    ...
)
LIKE my_catalog.my_db.Orders;

Dongwon Kim <[hidden email]> 于2020年11月11日周三 下午2:53写道:
Hi,

Is it disallowed to refer to a table from different databases or catalogs when someone creates a table?

According to [1], there's no way to refer to tables belonging to different databases or catalogs.

[1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/sql/create.html#create-table

Best,

Dongwon


--

Best,
Benchao Li