cccs-eric opened a new issue, #14491:
URL: https://github.com/apache/iceberg/issues/14491

   ### Apache Iceberg version
   
   1.9.1
   
   ### Query engine
   
   Spark
   
   ### Please describe the bug 🐞
   
   I am using a Polaris catalog with vended credentials enabled and 
encountering an issue when dropping tables with the `PURGE` option enabled. 
This happens when using Spark 3.5 and Iceberg 1.9.1.
   
   The SQL command I run is as follows:
   ```sql
   -- Using Spark SQL
   DROP TABLE xyz PURGE
   ```
   When examining the Spark HTTP requests sent to Polaris during this 
operation, I see the following sequence:
   
   1. `DELETE /api/catalog/v1/my_catalog/namespaces/my_ns/tables/my_tbl 
HTTP/1.1` returns `HTTP/1.1 204 No Content`
   
   2. `GET 
/api/catalog/v1/my_catalog/namespaces/my_ns/tables/my_tbl/credentials HTTP/1.1` 
returns `HTTP/1.1 404 Not Found` and the following body:
   ```json
   {"error":{"message":"Table does not exist: 
[my_ns.my](http://my_ns.my/)_tbl","type":"NoSuchTableException","code":404}}
   ```
   
   The table is successfully deleted from Polaris, but Spark then fails to 
delete the table's files because the credentials for the table cannot be 
retrieved. Since the table has already been deleted from Polaris, the GET 
request for its credentials results in an `HTTP 404 Not Found` error.
   
   Below is a truncated stack trace from my logs:
   ```
   ERROR   2025-10-29T17:48:55,975 154966  
com.azure.storage.common.policy.RequestRetryPolicy      [Thread-4]      Unable 
to process: Table does not exist: my_ns.my_tbl
   org.apache.iceberg.exceptions.RESTException: Unable to process: Table does 
not exist: my_ns.my_tbl
           at 
org.apache.iceberg.rest.ErrorHandlers$DefaultErrorHandler.accept(ErrorHandlers.java:248)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.rest.ErrorHandlers$DefaultErrorHandler.accept(ErrorHandlers.java:212)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.rest.HTTPClient.throwFailure(HTTPClient.java:215) 
~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at org.apache.iceberg.rest.HTTPClient.execute(HTTPClient.java:299) 
~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.rest.BaseHTTPClient.get(BaseHTTPClient.java:77) 
~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider.fetchCredentials(VendedAdlsCredentialProvider.java:139)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider.sasTokenForAccount(VendedAdlsCredentialProvider.java:82)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider.lambda$credentialForAccount$0(VendedAdlsCredentialProvider.java:75)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           ...
           at 
org.apache.iceberg.azure.adlsv2.VendedAdlsCredentialProvider.credentialForAccount(VendedAdlsCredentialProvider.java:78)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy.maybeUpdateCredential(VendedAzureSasCredentialPolicy.java:58)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.VendedAzureSasCredentialPolicy.processSync(VendedAzureSasCredentialPolicy.java:53)
 ~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
com.azure.core.http.HttpPipelineNextSyncPolicy.processSync(HttpPipelineNextSyncPolicy.java:53)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           at 
com.azure.storage.common.policy.MetadataValidationPolicy.processSync(MetadataValidationPolicy.java:42)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           ...
           at 
com.azure.storage.file.datalake.DataLakePathClient.lambda$existsWithResponse$10(DataLakePathClient.java:1593)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           at 
com.azure.storage.file.datalake.implementation.util.DataLakeImplUtils.returnOrConvertException(DataLakeImplUtils.java:51)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           at 
com.azure.storage.file.datalake.DataLakePathClient.existsWithResponse(DataLakePathClient.java:1593)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           at 
com.azure.storage.file.datalake.DataLakePathClient.exists(DataLakePathClient.java:1570)
 ~[iceberg-azure-bundle-1.9.1.jar:?]
           at 
org.apache.iceberg.azure.adlsv2.BaseADLSFile.exists(BaseADLSFile.java:66) 
~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
           at 
org.apache.iceberg.spark.SparkCatalog.purgeTable(SparkCatalog.java:372) 
~[iceberg-spark-runtime-3.5_2.12-1.9.1.0-SNAPSHOT.jar:?]
   ```
   
   The code responsible for this trace is 
[here](https://github.com/apache/iceberg/blob/apache-iceberg-1.9.1/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java#L372).
   
   Is this issue due to a fundamental limitation when using vended credentials? 
To me, this behavior appears broken for vended credentials because credential 
retrieval happens after the table is already dropped from the catalog. The 
`HTTP 404` response seems legitimate in this case.
   
   ### Willingness to contribute
   
   - [ ] I can contribute a fix for this bug independently
   - [ ] I would be willing to contribute a fix for this bug with guidance from 
the Iceberg community
   - [ ] I cannot contribute a fix for this bug at this time


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to