You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/07/06 07:50:51 UTC

[GitHub] [iceberg] JingsongLi opened a new issue #1170: Integrate Iceberg catalog to Flink catalog

JingsongLi opened a new issue #1170:
URL: https://github.com/apache/iceberg/issues/1170


   Like Spark 3, Flink also has `Catalog` interface, we can integrate Iceberg catalog to Flink catalog, iceberg as a Flink catalog, users can use Flink DDLs to manipulate iceberg metadata. And query iceberg tables directly.
   
   The biggest incompatible thing is the database and namespace.
   - In Flink: like Hive, the identifier of a table is `catalogName.databaseName.tableName`. The database name must exist, and it should be a single string (Not null or whitespace only).
   - In Iceberg, the identifier of a table is `namespace_level1.namespace_level2....tableName` in a catalog. The level number of namespace can be zero, one and more.
   
   A simple way is only support single namespace for Flink catalog, but this is too limited.
   
   A way can be:
   - For empty namespace: We can provide a config option `empty.namespace.name`, its default value can be `__DEFAULT_EMPTY_NAMESPACE__`.
   - For multi levels namespace: Using `Namespace.toString` as database name. In Flink SQL, supports quoting identifier. For example, using `iceberg_catalog.{quote}namespace_level1.namespace_level2{quote}.table_name`.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-654547070


   Thanks @rdblue for your responding. Very useful. Agree with you.
   - Database and namespace: base namespace for a given catalog, looks good to me.
   - Partitions: I mean DDL support. Only support identity partitions is too limited, so we don't need to use Flink DDL to express iceberg partition.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-654555451


   BTW, Empty `Namespace` is not legal namespace, right?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-655235848


   > This is fine in the Hadoop catalog, since it is just a directory structure.
   
   Got it, I was misled by `HadoopCatalog.listTables(Namespace)`. `FlinkCatalog` can not support empty namespace at first.
   
   > I'm not sure I'm following what you mean. If identity partitioning is too limited, would you support partition expressions? Or just not allow partitioning in DDL at all?
   
   Not allow partitioning using Flink `partitioned by` DDL, I think we provide an option key `partitioning` to define iceberg partitioning. (This require string parser for `PartitionSpec`).
   
   I'll create a PR soon.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-654392625


   For partitioning, are you talking about DDL support or working with tables?
   
   When reading a table, the engine works with only the table columns. When writing, the engine needs to partition data into files correctly, but that configuration comes from Iceberg and it is not exposed through SQL. The purpose is to maintain a separation between the logical data and the physical layout.
   
   The only limitation Flink should have is what it can express in DDL statements. If you support Hive-like syntax with `PARTITIONED BY`, then Flink would only support creating tables with identity partitions. But there is no need to actually restrict what tables it can write to.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi closed issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi closed issue #1170:
URL: https://github.com/apache/iceberg/issues/1170


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-655143789


   > Empty Namespace is an illegal namespace, right?
   
   This is fine in the Hadoop catalog, since it is just a directory structure.
   
   > Only support identity partitions is too limited, so we don't need to use Flink DDL to express iceberg partition.
   
   I'm not sure I'm following what you mean. If identity partitioning is too limited, would you support partition expressions? Or just not allow partitioning in DDL at all?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi edited a comment on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
JingsongLi edited a comment on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-654555451


   BTW, Empty `Namespace` is an illegal namespace, right?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on issue #1170: Integrate Iceberg catalog to Flink catalog

Posted by GitBox <gi...@apache.org>.
rdblue commented on issue #1170:
URL: https://github.com/apache/iceberg/issues/1170#issuecomment-654390863


   Another option to consider is supplying a base namespace for a given catalog. So if I have a catalog that supports a 2-level namespace, I would supply the first level in the catalog configuration and the second level would be exposed as Flink databases.
   
   That's what we were going to do in Spark, before there was a push for arbitrary nesting. The nesting is nice because we can reference metadata tables using the table name as a namespace (like `catalog.db.table.history`).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org