nickdelnano opened a new issue, #10226:
URL: https://github.com/apache/iceberg/issues/10226
### Apache Iceberg version
1.4.3
### Query engine
Flink
### Please describe the bug 🐞
Hi,
I am observing issues when creating an Iceberg table with a Glue catalog
configured to use Lake Formation.
I see an integration test case for the issue I am experiencing so I will
explain the issue through this. I will include details about my use case with
Flink afterwards.
This issue is very similar to https://github.com/apache/iceberg/issues/6523
however I observe the issue is not fixed.
### TestLakeFormationMetadataOperations.java test testCreateTableSuccess
[Link to
test](https://github.com/apache/iceberg/blob/main/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java#L167)
This test fails in my AWS account. I have walked through the code line by
line in a debugger and believe that it would fail in any environment due to the
below.
The test fails on this
[line](https://github.com/apache/iceberg/blob/main/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java#L182)
because Lake Formation permissions cannot be granted on a table that does not
exist. It first yields an exception from the call to
`glueCatalogPrivilegedRole.createTable` but then proceeds to the `finally`
block.
As far as I can tell, the AWS integration tests are not run on opened PRs so
I cannot easily demonstrate this in an issue or PR. If it is possible to do
this please let me know how and I will create a PR that shows it.
Previous work has been done to create an initial or "dummy" Glue table if
Lake Formation is enabled and the table requested for creation does not exist
yet ([1] https://github.com/apache/iceberg/pull/4423/files). However, if Lake
Formation is enabled, [2] [GlueCatalog sets
`put(S3FileIOProperties.PRELOAD_CLIENT_ENABLED,
String.valueOf(true)`](https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java#L224-L232),
which triggers the below code path and results in a call to `aws glue
get-table` API before any table exists. This causes an uncaught exception and
creating a table fails.
-
https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java#L374-L376
-
https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java#L78-L79
-
https://github.com/apache/iceberg/blob/main/aws/src/main/java/org/apache/iceberg/aws/lakeformation/LakeFormationAwsClientFactory.java#L107
Please provide any advice or workaround for how a table can be created in a
Glue catalog with Lake Formation enabled without encountering this issue.
### Error in my Flink environment
I am using Flink on EC2 (not EMR) and using Iceberg, Glue and Lake Formation.
Iceberg catalog configuration:
```
"CREATE CATALOG glue_catalog WITH (
'type'='iceberg',
'warehouse'='s3://bucket'
'catalog-impl'='org.apache.iceberg.aws.glue.GlueCatalog'
'io-impl'='org.apache.iceberg.aws.s3.S3FileIO'
'client.factory'='org.apache.iceberg.aws.lakeformation.LakeFormationAwsClientFactory'
'client.assume-role.arn'='arn:aws:iam:::role/'
'glue.lakeformation-enabled'='true'
'client.assume-role.tags.LakeFormationAuthorizedCaller'=''
'client.assume-role.region'='us-east-1'
'glue.account-id'=''
);
```
The stacktrace confirms the behavior explained for the integration test: in
the call stack of creating a table, `S3FileIO` is initialized and
`LakeFormationAwsClientFactory.isTableRegisteredWithLakeFormation` is called
before any Glue table exists.
Stacktrace:
```
Caused by:
software.amazon.awssdk.services.glue.model.EntityNotFoundException: Entity Not
Found (Service: Glue, Status Code: 400, Request ID:
efa126e5-e9d5-41f8-bb5a-c8d30bd166eb)
at
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleErrorResponse(CombinedResponseHandler.java:125)
~[?:?]
at
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handleResponse(CombinedResponseHandler.java:82)
~[?:?]
at
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:60)
~[?:?]
at
software.amazon.awssdk.core.internal.http.CombinedResponseHandler.handle(CombinedResponseHandler.java:41)
~[?:?]
at
software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:40)
~[?:?]
at
software.amazon.awssdk.core.internal.http.pi