apache / iceberg

Apache Iceberg
https://iceberg.apache.org/
Apache License 2.0
5.87k stars 2.06k forks source link

AWS: Creating a Glue table with Lake Formation enabled fails #10226

Open nickdelnano opened 2 months ago

nickdelnano commented 2 months ago

Apache Iceberg version

1.4.3

Query engine

Flink

Please describe the bug 🐞

Hi,

I am unable to create an Iceberg table with a Glue catalog configured to use Lake Formation. I am using Flink but the issue is reproduced with an existing integration test that fails when I run it in my AWS account. I have not seen how to run the AWS integration tests on the project CI but I assume it will fail there too.

This issue is similar to https://github.com/apache/iceberg/issues/6523 however I observe the issue is not fixed.

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::<redacted>:role/<redacted>'
                'glue.lakeformation-enabled'='true'
                'client.assume-role.tags.LakeFormationAuthorizedCaller'='<redacted>'
                'client.assume-role.region'='us-east-1'
                'glue.account-id'='<redacted>'
);

And then I create a table with Flink SQL. In the call stack of creating a table, S3FileIO is initialized and LakeFormationAwsClientFactory.isTableRegisteredWithLakeFormation is called before any Glue table exists and an uncaught exception is thrown since the table does not exist.

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.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:30) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:73) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptTimeoutTrackingStage.execute(ApiCallAttemptTimeoutTrackingStage.java:42) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:78) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.TimeoutExceptionHandlingStage.execute(TimeoutExceptionHandlingStage.java:40) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:50) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:36) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:81) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.RetryableStage.execute(RetryableStage.java:36) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) ~[?:?]
        at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:56) ~[?:?]
        at software.amazon.awssdk.core.internal.http.StreamManagingStage.execute(StreamManagingStage.java:36) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.executeWithTimer(ApiCallTimeoutTrackingStage.java:80) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:60) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallTimeoutTrackingStage.execute(ApiCallTimeoutTrackingStage.java:42) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:48) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:31) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.RequestPipelineBuilder$ComposingRequestPipelineStage.execute(RequestPipelineBuilder.java:206) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:37) ~[?:?]
        at software.amazon.awssdk.core.internal.http.pipeline.stages.ExecutionFailureExceptionReportingStage.execute(ExecutionFailureExceptionReportingStage.java:26) ~[?:?]
        at software.amazon.awssdk.core.internal.http.AmazonSyncHttpClient$RequestExecutionBuilderImpl.execute(AmazonSyncHttpClient.java:193) ~[?:?]
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103) ~[?:?]
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:171) ~[?:?]
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$1(BaseSyncClientHandler.java:82) ~[?:?]
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:179) ~[?:?]
        at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:76) ~[?:?]
        at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:45) ~[?:?]
        at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:56) ~[?:?]
        at software.amazon.awssdk.services.glue.DefaultGlueClient.getTable(DefaultGlueClient.java:8903) ~[?:?]
        at org.apache.iceberg.aws.lakeformation.LakeFormationAwsClientFactory.isTableRegisteredWithLakeFormation(LakeFormationAwsClientFactory.java:115) ~[?:?]
        at org.apache.iceberg.aws.lakeformation.LakeFormationAwsClientFactory.s3(LakeFormationAwsClientFactory.java:79) ~[?:?]
        at org.apache.iceberg.aws.s3.S3FileIO.client(S3FileIO.java:327) ~[?:?]
        at org.apache.iceberg.aws.s3.S3FileIO.initialize(S3FileIO.java:375) ~[?:?]
        at org.apache.iceberg.CatalogUtil.loadFileIO(CatalogUtil.java:325) ~[?:?]
        at org.apache.iceberg.aws.glue.GlueTableOperations.initializeFileIO(GlueTableOperations.java:223) ~[?:?]
        at org.apache.iceberg.aws.glue.GlueTableOperations.io(GlueTableOperations.java:115) ~[?:?]
        at org.apache.iceberg.aws.glue.GlueCatalog.newTableOps(GlueCatalog.java:246) ~[?:?]
        at org.apache.iceberg.BaseMetastoreCatalog$BaseMetastoreCatalogTableBuilder.create(BaseMetastoreCatalog.java:188) ~[?:?]
        at org.apache.iceberg.CachingCatalog$CachingTableBuilder.lambda$create$0(CachingCatalog.java:261) ~[?:?]
        at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2406) ~[?:?]
        at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1908) ~[?:?]
        at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2404) ~[?:?]
        at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2387) ~[?:?]
        at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:108) ~[?:?]
        at org.apache.iceberg.shaded.com.github.benmanes.caffeine.cache.LocalManualCache.get(LocalManualCache.java:62) ~[?:?]
        at org.apache.iceberg.CachingCatalog$CachingTableBuilder.create(CachingCatalog.java:257) ~[?:?]
        at org.apache.iceberg.catalog.Catalog.createTable(Catalog.java:75) ~[?:?]
        at org.apache.iceberg.flink.FlinkCatalog.createIcebergTable(FlinkCatalog.java:415) ~[?:?]
        at org.apache.iceberg.flink.FlinkCatalog.createTable(FlinkCatalog.java:394) ~[?:?]
        at org.apache.flink.table.catalog.CatalogManager.lambda$createTable$11(CatalogManager.java:663) ~[flink-table-api-java-uber-1.17.0.jar:1.17.0]
        at org.apache.flink.table.catalog.CatalogManager.execute(CatalogManager.java:909) ~[flink-table-api-java-uber-1.17.0.jar:1.17.0]
        at org.apache.flink.table.catalog.CatalogManager.createTable(CatalogManager.java:652) ~[flink-table-api-java-uber-1.17.0.jar:1.17.0]
        at org.apache.flink.table.api.internal.TableEnvironmentImpl.createTable(TableEnvironmentImpl.java:532) ~[flink-table-api-java-uber-1.17.0.jar:1.17.0]
        at 

Integration test that shows the issue

Test testCreateTableSuccess in TestLakeFormationMetadataOperations.java --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. As far as I can tell the AWS integration tests are not run on opened PRs so I cannot easily demonstrate on a public Github post. If it is possible please let me know how and I can do that.

The test fails on this line 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.

Previous work has been done to create a temporary Glue table if Lake Formation is enabled and the table requested for creation does not exist yet (https://github.com/apache/iceberg/pull/4423/files). However, if Lake Formation is enabled, GlueCatalog sets put(S3FileIOProperties.PRELOAD_CLIENT_ENABLED, String.valueOf(true), 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.

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.

Additional info

PRELOAD_CLIENT_ENABLED documentation (https://github.com/apache/iceberg/blob/10ffc606219d34c801c2109a9d19d0848a63d2dc/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java#L371-L375):

Albertagamergod1 commented 2 months ago

Continue your work please

nickdelnano commented 3 weeks ago

@singhpk234 @jackye1995 @xiaoxuandev (authors of previous lake formation PRs), could you please check this issue