Skip to content

Refactor Iceberg connector to support Iceberg native catalogs#16612

Merged
beinan merged 1 commit intoprestodb:masterfrom
jackye1995:multi-catalog
Oct 5, 2021
Merged

Refactor Iceberg connector to support Iceberg native catalogs#16612
beinan merged 1 commit intoprestodb:masterfrom
jackye1995:multi-catalog

Conversation

@jackye1995
Copy link
Copy Markdown

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 nativeMode that user can turn on in order to use a different ConnectorMetadata implementation called IcebergNativeMetadata. 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 the presto-hive related dependencies and deprecate the old implementations.

Test plan: added distributed and smoke tests running using Iceberg's HadoopCatalog, all tests pass.

== RELEASE NOTES ==
Iceberg Changes
* Iceberg connector now supports a native mode that can be used without a Hive installation to run queries against Iceberg  native catalogs.

cc @pettyjamesm

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The rollback code does nothing anyway, so I just removed it.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just wanna double confirm, both the rollback of IcebergMetadata and IcebergNativeMetadata do nothing?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FileIO related caching is not added yet, just to keep the initial implementation simple.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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 beinan self-assigned this Aug 17, 2021
Copy link
Copy Markdown
Member

@beinan beinan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, I will change to cached-catalog-num instead, using - to match the convention of Presto config keys.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just wanna double confirm, both the rollback of IcebergMetadata and IcebergNativeMetadata do nothing?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

good point, this was mostly following the structure of IcebergMetadata, let me update.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sure, I moved it together with the default case with a comment.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

"return" is not necessary I think. can we just write ".map(column -> new ColumnMetadata(column.name(), toPrestoType(column.type(), typeManager), column.doc(), false))"

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

@ChunxuTang ChunxuTang left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@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!

@jackye1995
Copy link
Copy Markdown
Author

@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.

@beinan
Copy link
Copy Markdown
Member

beinan commented Oct 4, 2021

rerun the test jobs

Copy link
Copy Markdown
Member

@beinan beinan left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm, great contribution for the presto+iceberg use cases. thanks a lot!

@beinan
Copy link
Copy Markdown
Member

beinan commented Oct 4, 2021

@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.

@jackye1995 could you rebase master again? looks like some of the tests are still failing. Thank you!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

iceberg Apache Iceberg related

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants