-
Notifications
You must be signed in to change notification settings - Fork 3k
AWS: Add LakeFormation credential support for GlueCatalog #4280
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
Thanks for putting this up! cc: @arminnajafi @rajarshisarkar @singhpk234 @amogh-jahagirdar @yyanyy @natsukawa-kanou |
| * | ||
| * @return this table's parameters map | ||
| */ | ||
| Map<String, String> parameters(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks like the addition of parameters is for injecting the LF authorized column information. I think we can reduce the scope of this PR a little bit, to first offer only database and table level permission management.
I understand we cannot leverage properties because it will be written back in a commit if we add additional information. But adding another parameters is definitely confusing, and we should explore some better ways. It would be much clearer to discuss after we first add database and table level LF support with the existing Iceberg interfaces.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I really doubt that we should be changing the table API for this.
| this.hadoopConf = conf; | ||
| } | ||
|
|
||
| protected LockManager getLockManager() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
prefer to not have get, e.g. lockManager() instead of getLockManager()
| import org.apache.htrace.fasterxml.jackson.core.JsonFactory; | ||
| import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper; | ||
|
|
||
| public class LakeFormationAuthorizedColumns { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As commented before, we can first remove column-level support in this PR to be more focused
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This looks like it goes beyond configuring a credentials provider. What is the goal that you're trying to support by adding classes like this?
| private String secretAccessKey; | ||
| private String sessionToken; | ||
|
|
||
| /** |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: no need for this javadoc
| return AwsSessionCredentials.create(response.accessKeyId(), response.secretAccessKey(), response.sessionToken()); | ||
| } | ||
|
|
||
| private List<String> getPrivilegeList() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this looks like test code?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes, I add this simple resolveCredentials to reduce complexity, will add cache later.
| import software.amazon.awssdk.services.lakeformation.model.GetTemporaryGlueTableCredentialsRequest; | ||
| import software.amazon.awssdk.services.lakeformation.model.GetTemporaryGlueTableCredentialsResponse; | ||
|
|
||
| public class LakeFormationCredentialsProvider implements AwsCredentialsProvider { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we can move this as an inner class in LakeFormationAwsClientFactory
| private LakeFormationCatalogProperties() { | ||
| } | ||
|
|
||
| public static final String LAKE_FORMATION_AUTHORIZED_COLUMNS_KEY = "___lake_formation_authorized_columns___"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
As commented before, we can first remove column-level support in this PR to be more focused
|
@xiaoxuandev can you add details? What does it mean to support LakeFormation? |
|
@rdblue for more details, AWS Glue and LakeFormation recently released the public APIs for third-party integration of using AWS LakeFormation to manage database, table and column level access control. For database and table level access control, the integration is mostly through configuring the correct AWS credential provider to access Glue and S3. See https://docs.aws.amazon.com/lake-formation/latest/dg/api-overview.html for more details. |
|
@jackye1995 thanks for the additional detail. Adding a way to configure the credential provider seems reasonable to me. More broadly though, Iceberg's purpose so far has been to standardize storage and provide tools for reading and writing that storage. Access control is orthogonal to that, so I've not thought there was a good use case for mixing access control enforcement into the Iceberg library. Plus enforcement can't really be done client side. So it's good to hear that the integration is just configuring the client properly. |
| public void initialize(Map<String, String> catalogProperties) { | ||
| this.properties = Maps.newHashMap(catalogProperties); | ||
| if (properties.containsKey(LakeFormationCatalogProperties.USE_LAKE_FORMATION_CREDENTIALS) && | ||
| properties.get(LakeFormationCatalogProperties.USE_LAKE_FORMATION_CREDENTIALS) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Indentation in this file isn't correct. It should be 4 spaces, not 8.
| } | ||
| } | ||
|
|
||
| private static Region getRegion(Map<String, String> properties) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Even for private methods, avoid using get. It is either unnecessary or should be replaced with a more specific verb.
| public static final String DB_NAME_KEY = "_lake_formation_db_name_"; | ||
| public static final String AWS_REGION = "_lake_formation_aws_region_"; | ||
| public static final String ACCOUNT_ID = "_lake_formation_account_id_"; | ||
| public static final String QUERY_ID = "_lake_formation_query_id_"; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think these should conform to the conventions that are used for other catalogs.
| import software.amazon.awssdk.services.glue.model.GetUnfilteredTableMetadataRequest; | ||
| import software.amazon.awssdk.services.glue.model.GetUnfilteredTableMetadataResponse; | ||
|
|
||
| public class LakeFormationCatalog extends GlueCatalog { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why does this require a separate catalog implementation? It looks like this is to add authorized columns and credentials in loadTable. Instead, I think those should be implementation details in the TableOperations. You can pass them separately that way.
rdblue
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think this should change the Table API. I'm also curious about the long-term plan for access control integration because I don't think we want to put very many features in Iceberg for it.
Yes agree, please hold on the review for some time, I am asking for an initial draft from @xiaoxuandev so we can just openly review the code changes. I think this needs a few iterations to trim down the changes. We should be able to make this feature work through just creating the correct TableOperation object using the correct credentials. I don't even think we need a
I think database and table level access control is most likely going to be all we add in Iceberg. For column-level and cell-level access control, all the AWS compute platforms are mostly already onboard with that, and there is guidelines published now for any third-party integration, it's not something that could be enforced just within the table format itself. |
No problem. I hope it wasn't a nuisance that I reviewed earlier.
Why put this in Iceberg at all? If it can't be enforced then is it worth complication? Wouldn't this just open up the project to other ineffective integrations? |
The fact is that Glue and LakeFormation are 2 very tightly coupled products, around 70% of customers using The LakeFormation permission control consists of 2 parts, (1) normal AWS credential is used to check access against Glue based on IAM + LakeFormation policy, (2) LF-vended temporary credential is used to access S3. When a user enrolls in LakeFormation (which happens when the user visits the LakeFormation console), the Glue access control automatically changes respect LakeFormation policy because of (1). This works fine for Hive tables because data and catalog are operated separately, but for Iceberg table all of a sudden users can no longer write and commit to the table because of (2). I would say this is more like a bug of |
|
I think adding credentials makes sense. You just mentioned table-level access control. Maybe I misinterpreted what you meant by that. I thought you meant some sort of enforcement or integration beyond user identity or credentials. I don't think it makes sense to add anything to Iceberg beyond that. |
Yes agree, the original PR also included changes for column level security integration which I think is too much, so I am already cutting the scope. That's a primary reason I am asking for direct open source review, because I feel the internal development is diverging a bit too much and it's better if we just publish openly to discuss the right scope within Iceberg, so thanks very much for the input here! |
eca0cd3 to
ccaa7f8
Compare
aws/src/main/java/org/apache/iceberg/aws/LakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
aws/src/main/java/org/apache/iceberg/aws/LakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
aws/src/main/java/org/apache/iceberg/aws/LakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
ccaa7f8 to
0c729e0
Compare
jackye1995
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for all the work! This looks good to me, @rdblue let me know if you have any further comments, otherwise I think everything is ready for merge.
aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java
Outdated
Show resolved
Hide resolved
72c377e to
59eb1e4
Compare
...integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
...integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java
Show resolved
Hide resolved
amogh-jahagirdar
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall looks good to me. Thanks Xiaoxuan for the contribution!
singhpk234
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @xiaoxuandev for this feature :) !!!
The change mostly LGTM
aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java
Show resolved
Hide resolved
...integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
...integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java
Outdated
Show resolved
Hide resolved
f19f377 to
24a117a
Compare
24a117a to
123a08e
Compare
|
@rdblue I am merging this because there is a subsequent PR for some caching improvements to the codepath. I think this has been reduced to what we have discussed. The LF integration is only a new client factory, and there is no change to the codebase outside GlueCatalog. Let us know if you have any additional concern, and we can discuss in subsequent PRs. Thanks @xiaoxuandev for bearing with all my nitpickings, this is very important contribution! |
| .put(AwsProperties.LAKE_FORMATION_DB_NAME, | ||
| IcebergToGlueConverter.getDatabaseName(tableIdentifier)) | ||
| .put(AwsProperties.LAKE_FORMATION_TABLE_NAME, | ||
| IcebergToGlueConverter.getTableName(tableIdentifier)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@xiaoxuandev and @jackye1995, I agree with the comment above that this seems confusing. If catalog properties are non-null, then lake formation logic is applied. I think at least the addition of these properties should be conditional on lake formation enabled.
|
@jackye1995 @xiaoxuandev I went looking for the caching you mentioned but came up empty. We're starting to use the Lake Formation support in Iceberg, noticing it's very chatty with the credential vending API, and a little concerned about scalability in terms of writing to a large number of tables frequently. Any pointers? Thanks! |
|
We added one feature for https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java#L652, which should to some extent mitigate the issue. When it pre-fetches a S3 client, it already serves as a cache for the LF-vended credentials. The issue with this is that when the FileIO is sent across nodes, the credential might be lost. The full solution to distribute the credential for reuse across nodes is not developed yet. |
|
It looks like a new Thanks! |
|
For anyone else winding up here, I wrapped out catalog in |
AWS LakeFormation provides a central place where users can grant data/metadata security permissions such as access/update/delete databases, tables as well as columns and rows to other users. This change is focus on adding db and table level access control.
Find more information on How to use LakeFormation Credential vending APIs