Refactor Iceberg connector to support Iceberg native catalogs#16612
Refactor Iceberg connector to support Iceberg native catalogs#16612beinan merged 1 commit intoprestodb:masterfrom
Conversation
There was a problem hiding this comment.
The rollback code does nothing anyway, so I just removed it.
There was a problem hiding this comment.
Just wanna double confirm, both the rollback of IcebergMetadata and IcebergNativeMetadata do nothing?
There was a problem hiding this comment.
Iceberg in general uses the pattern of dynamic plugin, users can plugin their own implementations of Catalog, FileIO and a couple of other important classes. If this is an anti-pattern for Presto, we can move to a Enum type. This feature is mostly oriented for customization of enterprise users, they also need to load in their own jar containing the implementation class during installation time.
There was a problem hiding this comment.
I might prefer to move it to an enum, we can config it in the config file of the presto-iceberg connector, and then initialized the catalog impl in a module. Let me know if you wanna see some similar examples in presto.
There was a problem hiding this comment.
This is basically to get a string representation of the user identity. As long as the identity does not change, the same catalog implementation can be used. This technically should not need any cache expiration, because the Catalog instance itself does not check access, but when any catalog action is called the access of the identity is checked, so the cache can still work when the access policy of the identity changes at server side.
There was a problem hiding this comment.
FileIO related caching is not added yet, just to keep the initial implementation simple.
There was a problem hiding this comment.
although this is reloading the Iceberg table, because we always input the actual snapshot ID, snapshot isolation is still guaranteed. Some optimizations can be made to cache the table object, but I am not adding that here to keep things simple.
beinan
left a comment
There was a problem hiding this comment.
Awesome PR! I heard quite a few giant companies are waiting for this one. Really appreciate your contribution! Just put a few comments, I will continue review it tomorrow.
There was a problem hiding this comment.
I might prefer to move it to an enum, we can config it in the config file of the presto-iceberg connector, and then initialized the catalog impl in a module. Let me know if you wanna see some similar examples in presto.
There was a problem hiding this comment.
this name is a little bit vague, cache-size sounds like the number of bytes. thinking if we can use cached_catalog_num or something? your call.
There was a problem hiding this comment.
sure, I will change to cached-catalog-num instead, using - to match the convention of Presto config keys.
There was a problem hiding this comment.
Just wanna double confirm, both the rollback of IcebergMetadata and IcebergNativeMetadata do nothing?
There was a problem hiding this comment.
I know we're doing something similar every where -- return a null when we got NoSuchTableException. Thinking if we could put a comment to explain why we return a null rather than throw an exception?
There was a problem hiding this comment.
I think returning null if table not exist in getTableHandle seems like the consistent behavior across all connectors, so it's probably fine to not give an explanation.
There was a problem hiding this comment.
why do we need extract getRawSystemTable as a separate private method? shall we just inline the code into getSystemTable? I didn't see any other caller of this method.
There was a problem hiding this comment.
good point, this was mostly following the structure of IcebergMetadata, let me update.
There was a problem hiding this comment.
nit: we do not need this case actually. but I'm ok if you would like put it explicitly here -- add a comment to explain why we do not have system table for DATA case?
There was a problem hiding this comment.
sure, I moved it together with the default case with a comment.
There was a problem hiding this comment.
nit: hmmm, I'm not sure if we should set a null or use orElse("") here. why do you use an empty string rather than null?
There was a problem hiding this comment.
good point, this is to match the behavior of IcebergMetadata, but I agree it seems awkward to return empty string. That constructor is deprecated anyway, I will switch to use the builder.
There was a problem hiding this comment.
"return" is not necessary I think. can we just write ".map(column -> new ColumnMetadata(column.name(), toPrestoType(column.type(), typeManager), column.doc(), false))"
There was a problem hiding this comment.
Similar to the last one, this is to match the behavior of IcebergMetadata, but that constructor is deprecated anyway, I will switch to use the builder.
d18743f to
d6b2b8e
Compare
ChunxuTang
left a comment
There was a problem hiding this comment.
@jackye1995 Thanks so much for your work! Generally, the PR looks good to me. We can iterate on the work here for further improvement.
I noticed that there're some compilation errors. At your convenience, could you fix them and re-run the tests? Thanks!
d6b2b8e to
e6e5986
Compare
|
@ChunxuTang Thanks for the review! The error was due to rebase, I have fixed it and ran tests locally to make sure it passes. I have also fixed based on your comments. |
|
rerun the test jobs |
beinan
left a comment
There was a problem hiding this comment.
lgtm, great contribution for the presto+iceberg use cases. thanks a lot!
@jackye1995 could you rebase master again? looks like some of the tests are still failing. Thank you! |
e6e5986 to
69e9906
Compare
Had some offline discussions with the maintainers of the Iceberg connector @zhenxiao @ChunxuTang @beinan , the direction that the community would like to go is to gradually remove its Hive dependency and make a pure Iceberg connector that can evolve independently.
Based on this ask, this PR introduces a
nativeModethat user can turn on in order to use a differentConnectorMetadataimplementation calledIcebergNativeMetadata. This implementation has no dependency on the Hive connector. The plan is to first develop all the features using this switch, and once it is feature complete and stable, we can drop all thepresto-hiverelated dependencies and deprecate the old implementations.Test plan: added distributed and smoke tests running using Iceberg's
HadoopCatalog, all tests pass.cc @pettyjamesm