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

Denys Kuzmenko commented on HIVE-28658:
---------------------------------------

1. Therefore, i think this ticket does't have any conflict with PR-5145. Am i 
right? 
I think there might be some conflicts since `HiveCatalog` was refactored. 
No idea why there is a coupling on hive-iceberg-handler in 
`hive-metastore-icecat` instead of iceberg-hive-metastore. HMS is supposed to 
be a standalone project and shouldn't depend on query-engine modules, instead 
could import native iceberg libs directly; 

2.  We need this ticket to make Hive as a query engine more catalog-neutral. 
Exactly. Would be nice if we could configure catalogs on db/table level, but 
that would require transforming HMS into a multi-catalog (registry of 
catalogs). 
Maybe at this moment, we could limit catalog definition to a table type, 
similar to Trino

iceberg.catalog.type=rest
iceberg.rest-catalog.uri=https://iceberg-with-rest:8181/ 

`class HiveCatalog extends BaseMetastoreCatalog` is a client abstraction used 
by query-engine (HS2). Under the hood, it uses a client pool 
`ClientPool<IMetaStoreClient, TException>` to communicate with the actual 
catalog (currently just HMS)

We need to make an Iface pluggable. I am leaning towards option 2 in the above 
doc as well.


>  Iceberg REST Catalog Support
> -----------------------------
>
>                 Key: HIVE-28658
>                 URL: https://issues.apache.org/jira/browse/HIVE-28658
>             Project: Hive
>          Issue Type: Improvement
>      Security Level: Public(Viewable by anyone) 
>          Components: Iceberg integration
>            Reporter: Denys Kuzmenko
>            Assignee: Zoltán Rátkai
>            Priority: Major
>
> Allow Hive to communicate directly with an Iceberg catalog service using the 
> Iceberg REST Catalog protocol. This protocol is the standard and recommended 
> way to interact with Iceberg metadata and tables.
> interface MetadataProvider
> * HiveCatalog
> * RestCatalog
> Ref: 
> https://www.tabular.io/apache-iceberg-cookbook/getting-started-trino-configuration/



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to