-
Notifications
You must be signed in to change notification settings - Fork 2.4k
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
Spark Iceberg REST Catalog refresh token #12363
Comments
@nqvuong1998 which version if Iceberg and Spark are you using?
|
Hi @c-thiel , Note: Trino and Spark used same clientId and clientSecret. Spark: 3.5.2 |
@nqvuong1998 I did some further tests and tracked it down. The problem is two-fold: iceberg/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java Lines 154 to 167 in 673192d
The second half of the problem is that Keycloak didn't implement the token exchange flow according to the RFC. Iceberg did. I opened an Issue for Keycloak exactly for this some time back: keycloak/keycloak#36203 It seems that trino took the easier route and just re-performs the initial client credential flow. @adutra do you know if the new Auth Manager would solve this? |
Hi @c-thiel and @nqvuong1998! Yes, the fact that token refreshes are broken when using an external IDP like Keycloak is a known issue. To summarize (and your analyzis was spot on!), there are two issues:
I posted some explanations here: #12196 (comment).
The AuthManager API can definitely help with this issue, although it won't solve it per se. Let's push to have it merged, and then I can move on and provide a fix for the token refresh problem: #12197 |
@c-thiel about this part of your comment:
So, here, I'm not sure I agree with you. Reading RFC 8693 section 2.1:
So my understanding is that token exchange is no different from other flows defined in RFC 6749 wrt client authentication. In particular, the actor token is NOT meant to represent the client, but rather:
So, in my opinion, Keycloak is rather correct here. It's Iceberg that is doing a malformed token exchange request. FYI Keycloak still considers token exchange in preview state. They have been collecting use cases for quite a while now under this issue, which is worth reading: |
We're experiencing similar problems with token refresh (Iceberg Rest Catalog with Trino and Spark) and Keycloak. And there is one interesting observation: |
Yes, in certain cases Keycloak may be able to refresh the token: Iceberg REST will try 5 times to refresh using bearer token, then will fall back to basic auth. So in certain cases, it is possible that the refresh eventually works with basic auth, but that is possible only if 1) the IDP supports token exchange and 2) the IDP is correctly configured to take the subject token and issue a new access token for it. |
@varpa89 I don't know what From what I see, your session cache is full of "context" sessions because only context sessions have UUID keys. Are you creating the catalog client with this constructor? In any case, you shouldn't end up with one auth session per request, session contexts are meant to be reused. |
@varpa89 I think I see where your problem could be coming from: if you are using these constructors: iceberg/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java Lines 55 to 63 in 1e126e2
And creating lots of catalog instances, then you would end up with lots of cached auth sessions. That's because I wonder if |
@adutra oh I see, sorry for the confusion. iceberg.rest-catalog.session is a Trino's property. And this is the place where they create a new SessionContext with random UUID. |
Yes it seems so; I would suggest using something deterministic to create the session context ID for the |
Query engine
Spark
Question
When I set up Spark to connect to the REST Catalog (Lakekeeper) using Keycloak as the OAuth2 server with the client credentials flow, I configured the token to expire after 5 minutes on Keycloak. Once the 5 minutes elapsed, the REST Catalog returned an "Unauthorized" exception. It appears that Spark's REST Catalog does not refresh the token from the OAuth2 server.
spark.sql.catalog.rest=org.apache.iceberg.spark.SparkCatalog
spark.sql.catalog.rest.type=rest
spark.sql.catalog.rest.uri=...
spark.sql.catalog.rest.credential={{ .Env.CLIENT_ID }}:{{ .Env.CLIENT_SECRET }}
spark.sql.catalog.rest.warehouse=my_warehouse
spark.sql.catalog.rest.scope=...
spark.sql.catalog.rest.oauth2-server-uri=...
spark.sql.catalog.rest.io-impl=org.apache.iceberg.aws.s3.S3FileIO
A similar setup on Trino works correctly.
The text was updated successfully, but these errors were encountered: