projectnessie / nessie

Nessie: Transactional Catalog for Data Lakes with Git-like semantics
https://projectnessie.org
Apache License 2.0
989 stars 130 forks source link

[Catalog] Unable to recover from running with invalid credentials #8860

Open dimas-b opened 3 months ago

dimas-b commented 3 months ago

Issue description

If the Catalog Server is started with invalid credentials and makes some attempts to fetch Iceberg metadata from storage, it remembers those failures and is unable to recover when it is restarted with valid credentials.


Use case:

1) Run Nessie Server with permanent Nessie storage (e.g. PosgreSQL) and invalid storage (e.g. S3) credentials

2) Run Spark using the basic Nessie Core API

$ bin/spark-sql \
 --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.5.0,org.apache.iceberg:iceberg-aws-bundle:1.5.0 \
 --conf spark.sql.catalog.nessie.uri=http://localhost:19120/api/v2 \
 --conf spark.sql.catalog.nessie.ref=main \
 --conf spark.sql.catalog.nessie.catalog-impl=org.apache.iceberg.nessie.NessieCatalog \
 --conf spark.sql.catalog.nessie.warehouse=s3://EXAMPLE \
 --conf spark.sql.catalog.nessie.io-impl=org.apache.iceberg.io.ResolvingFileIO \
 --conf spark.sql.catalog.nessie=org.apache.iceberg.spark.SparkCatalog \
  --master 'local[1]'

3) Create an Iceberg table

spark-sql (default)> create namespace nessie.ns
                   > ;
Time taken: 2.297 seconds
spark-sql (default)> create table nessie.ns.t3(x int);
Time taken: 1.68 seconds
spark-sql (default)> insert into nessie.ns.t3 values (123);
Time taken: 2.985 seconds
spark-sql (default)> select * from nessie.ns.t3;
123
Time taken: 1.6 seconds, Fetched 1 row(s)

4) Run Spark using the REST Catalog API

$ bin/spark-sql \
 --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.5.0,org.apache.iceberg:iceberg-aws-bundle:1.5.0 \
 --conf spark.sql.catalog.nessie.uri=http://localhost:19120/iceberg \
 --conf spark.sql.catalog.nessie.type=rest \
 --conf spark.sql.catalog.nessie=org.apache.iceberg.spark.SparkCatalog

5) Try to select from the test table:

spark-sql (default)> select * from nessie.ns.t3;
24/06/18 13:41:52 ERROR SparkSQLDriver: Failed in [select * from nessie.ns.t3]
org.apache.iceberg.exceptions.ServiceFailureException: Server error: IOException: Failed to read table metadata from s3://EXAMPLE/ns/t3_27cd8815-6e8a-4dc2-9398-37daf00b9a5e/metadata/00001-2a749105-bac1-4d4c-aafc-368164a4e264.metadata.json
    at org.apache.iceberg.rest.ErrorHandlers$DefaultErrorHandler.accept(ErrorHandlers.java:217)
    at org.apache.iceberg.rest.ErrorHandlers$TableErrorHandler.accept(ErrorHandlers.java:118)
    at org.apache.iceberg.rest.ErrorHandlers$TableErrorHandler.accept(ErrorHandlers.java:102)
    at org.apache.iceberg.rest.HTTPClient.throwFailure(HTTPClient.java:201)
    at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:313)
    at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:252)
    at org.apache.iceberg.rest.HTTPClient.get(HTTPClient.java:348)
    at org.apache.iceberg.rest.RESTClient.get(RESTClient.java:96)
    at org.apache.iceberg.rest.RESTSessionCatalog.loadInternal(RESTSessionCatalog.java:331)
    at org.apache.iceberg.rest.RESTSessionCatalog.loadTable(RESTSessionCatalog.java:347)
... etc ...

The Nessie Server gets this error:

2024-06-18 13:41:52,632 WARN  [org.pro.cat.ser.res.IcebergErrorMapper] (executor-thread-1) Unhandled exception returned as HTTP/500: java.io.IOException: Failed to read table metadata from s3://EXAMPLE/ns/t3_27cd8815-6e8a-4dc2-9398-37daf00b9a5e/metadata/00001-2a749105-bac1-4d4c-aafc-368164a4e264.metadata.json: java.io.IOException: Failed to read table metadata from s3://EXAMPLE/ns/t3_27cd8815-6e8a-4dc2-9398-37daf00b9a5e/metadata/00001-2a749105-bac1-4d4c-aafc-368164a4e264.metadata.json
    at org.projectnessie.catalog.service.impl.ImportSnapshotWorker.importIcebergTable(ImportSnapshotWorker.java:118)
    at org.projectnessie.catalog.service.impl.ImportSnapshotWorker.importSnapshot(ImportSnapshotWorker.java:68)
    at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
    at io.smallrye.context.impl.wrappers.SlowContextualRunnable.run(SlowContextualRunnable.java:19)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)
Caused by: org.projectnessie.catalog.files.api.NonRetryableException: software.amazon.awssdk.services.s3.model.S3Exception: The request signature we calculated does not match the signature you provided. Check your key and signing method. (Service: S3, Status Code: 403, Request ID: K3F48XFM5EVN26W7, Extended Request ID: FFbwyT+UQZjEXaWGPcX/6Xiw0ytRylZQCWPgnZqmxiaGyETA6jFw9q3Xi0jxCLKXafJ22nqSwqD9aKdl+himdg==)
    at org.projectnessie.catalog.files.s3.S3ObjectIO.readObject(S3ObjectIO.java:71)
    at org.projectnessie.catalog.files.DelegatingObjectIO.readObject(DelegatingObjectIO.java:34)
    at org.projectnessie.catalog.service.impl.ImportSnapshotWorker.importIcebergTable(ImportSnapshotWorker.java:111)
    ... 6 more
Caused by: software.amazon.awssdk.services.s3.model.S3Exception: The request signature we calculated does not match the signature you provided. Check your key and signing method. (Service: S3, Status Code: 403, Request ID: K3F48XFM5EVN26W7, Extended Request ID: FFbwyT+UQZjEXaWGPcX/6Xiw0ytRylZQCWPgnZqmxiaGyETA6jFw9q3Xi0jxCLKXafJ22nqSwqD9aKdl+himdg==)
    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:50)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.HandleResponseStage.execute(HandleResponseStage.java:38)
    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:72)
    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:55)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallAttemptMetricCollectionStage.execute(ApiCallAttemptMetricCollectionStage.java:39)
    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:50)
    at software.amazon.awssdk.core.internal.http.pipeline.stages.ApiCallMetricCollectionStage.execute(ApiCallMetricCollectionStage.java:32)
    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:224)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.invoke(BaseSyncClientHandler.java:103)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.doExecute(BaseSyncClientHandler.java:173)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:66)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.measureApiCallSuccess(BaseSyncClientHandler.java:182)
    at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.execute(BaseSyncClientHandler.java:60)
    at software.amazon.awssdk.core.client.handler.SdkSyncClientHandler.execute(SdkSyncClientHandler.java:52)
    at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.execute(AwsSyncClientHandler.java:60)
    at software.amazon.awssdk.services.s3.DefaultS3Client.getObject(DefaultS3Client.java:5203)
    at software.amazon.awssdk.services.s3.S3Client.getObject(S3Client.java:9063)
    at org.projectnessie.catalog.files.s3.S3ObjectIO.readObject(S3ObjectIO.java:57)
    ... 8 more

6) Restart the Nessie Server with correct S3 credentials and restart the Spark Session using the REST catalog

7) Select from the test table again. Spark gets the same error as above. Nessie Server gets the following error.

2024-06-18 13:43:28,402 ERROR [org.pro.nes.tas.ser.imp.TasksServiceImpl] (tasks-async-1) tasks: Unhandled state during local task attempt for catalog-snapshot:542bab2a3cf31ea7da84105d2302de7661208b8054591f2fd7c787eb9bcc35e6: java.lang.RuntimeException: java.util.concurrent.CompletionException: java.lang.RuntimeException: java.io.IOException: Failed to read table metadata from s3://EXAMPLE/ns/t3_27cd8815-6e8a-4dc2-9398-37daf00b9a5e/metadata/00001-2a749105-bac1-4d4c-aafc-368164a4e264.metadata.json
    at org.projectnessie.catalog.service.impl.EntitySnapshotTaskBehavior.stateAsException(EntitySnapshotTaskBehavior.java:37)
    at org.projectnessie.catalog.service.impl.EntitySnapshotTaskBehavior.stateAsException(EntitySnapshotTaskBehavior.java:29)
    at org.projectnessie.nessie.tasks.service.impl.TasksServiceImpl.tryLocal(TasksServiceImpl.java:200)
    at org.projectnessie.nessie.tasks.service.impl.TasksServiceImpl.lambda$submit$2(TasksServiceImpl.java:152)
    at io.smallrye.context.impl.wrappers.SlowContextualRunnable.run(SlowContextualRunnable.java:19)
    at org.projectnessie.nessie.tasks.async.TasksAsync.lambda$call$0(TasksAsync.java:29)
    at java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
    at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
    at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
    at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
    at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
    at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
    at java.base/java.lang.Thread.run(Thread.java:840)

Notice that the metadata is accessible now, but is not re-fetched.

8) To confirm access, create a new table.

spark-sql (default)> create table nessie.ns.t4(x int);
Time taken: 2.228 seconds