-
Notifications
You must be signed in to change notification settings - Fork 3.1k
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
Trino intermittently fails to pick up IRSA #15267
Comments
@Pluies I wonder what is being done in the |
@skyahead it doesn't even keep looking, it only does it once 😄 Well, actually, being a subclass of AWS' If it helps, it looks like this: package com.foo
import com.amazonaws.auth.WebIdentityTokenCredentialsProvider
import java.net.URI
import org.apache.hadoop.conf.Configuration
class CustomWebIdentityTokenCredentialsProvider(uri: URI, hadoopConf: Configuration) :
WebIdentityTokenCredentialsProvider() {} And our <configuration>
<property>
<name>trino.s3.credentials-provider</name>
<value>com.foo.CustomWebIdentityTokenCredentialsProvider</value>
<description>Custom IAM credentials provider to force IRSA</description>
</property>
</configuration> |
thanks so much, will give it a try |
I suspect the fact that If your code happens to run in an environment which has multiple set of credentials available (e.g. IRSA role from container and EC2 instance profile) and only one of them allows for S3 bucket's access, then what might be happening is the following:
|
cc: @pettyjamesm Have you by chance ever run into this or have ideas if something is wrong on the Trino side? |
We have a very different setup for credential management on our side because instances don’t typically use their own credentials to access AWS services- but I have seen the instance profile metadata endpoint return throttling exceptions or otherwise fail intermittently in some cases so presumably something similar could happen to the IRSA provider. If that were to happen, then @mccartney’s description of what could happen makes sense based on the code snippets linked. In general, it does seem like a bad idea to have a “chain” of credentials providers that can yield credentials with very different access permissions. Probably you want a way to specify to either use IRSA credentials or fail without trying to fall back to instance profile credentials. |
Got it. So we'd can add a config which when enabled would use just IRSA creds instead of default chain. cc: @electrum this is regarding better EKS integration |
cc @hashhar @pettyjamesm @mccartney @electrum We have also faced similar issues and endup implementing a custom credential provider, which internally uses We have also added extra logging, so that if anything breaks, we will get more information, which is an another problem when these issues happen in production, and there is not much to trace. Since we have pinned the credential provider to the IMO, It would be good to add a We are happy to contribute this to upstream as well, as looks like it will help other teams as well. thanks! |
For those who are experiencing the same problem in the production environment so need immediate fix. I Assume that the cause of this problem is the same as what other users discussed above: We need to fix the credential provider configuration to use (1) Glue Catalog: we can use (2) HDFS-S3 File System: Fix
(3, Optional) Exchange File System: Add a configuration like After the trino cluster is deployed, you can check whether credential provider chain is used by setting |
FYI: WebIdentityTokenFileCredentialsProvider is different w/ WebIdentityTokenCredentialsProvider
|
After the modification, the IRSA Access Denied Issue never happen. |
Hi team, any update on this, did we get this fixed in the later version? |
Hey, we came across this issue today too, we're on Trino 435. Is this currently being looked into? |
@1ambda would u be able to provider your config file? It's been quite hard to understand how to piece things together. |
This has also become an issue for me. I will work on finding a fix today, but it'd be nice if someone already had one. I have attached the output of |
Wondering, How we are fixing it?
We have fixed it by using the custom credential provider to pin it to use
WebIdentity instead of the chain, and make it default, and we did not face
that issue any more.
Regards,
Manish
…On Fri, Mar 1, 2024 at 6:05 AM Samuel Chai ***@***.***> wrote:
@1ambda <https://github.com/1ambda> would u be able to provider your
config file? It's been quite hard to understand how to piece things
together.
This has also become an issue for me. I will work on finding a fix today,
but it'd be nice if someone already had one.
helm_values.txt
<https://github.com/trinodb/trino/files/14462038/helm_values.txt>
—
Reply to this email directly, view it on GitHub
<#15267 (comment)>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AXQ2PYTQA3OUPZGDPYZLYHLYWCDMDAVCNFSM6AAAAAASQVC3CKVHI2DSMVQWIX3LMV43OSLTON2WKQ3PNVWWK3TUHMYTSNZTGI3DMMJRHE>
.
You are receiving this because you commented.Message ID:
***@***.***>
|
Is the solution simply to update configs to include Sorry, I'm a DevOps guy trying to help out the data team and have limited Java and no Trino knowledge. |
Hey all - I was following along for this issue as we ran into the same thing. I've now updated to version 451, and am using Not often, but occasionally my worker pods can still get into a bad state. Any queries running on those pods fail with the below error. Sure enough, restarting the pods seems to do the trick. I'm using IRSA to query parquet files in s3 buckets.
|
@mgorbatenko The |
Hey @rohanag12, apologies for the confusion. I am actually using 453. We upgraded from 451 -> 453 when this was released. I misspoke in my previous comment. |
Hello Trino folks!
We've been running Trino 403 on EKS via the Helm chart, with IRSA enabled to read Delta files from S3 buckets.
This setup works very well for us... Most of the time. Unfortunately, sometimes, a node will start up and fail to get its IRSA authentication, and will use its instance profile instead. The instance profile does not have the correct S3 permissions, and any calls to S3 will then fail with a 403 Access Denied, causing Trino queries to fail with the following stacktrace:
The only way to fix this issue is to delete the worker pod.
We confirmed via S3 logs that the problematic requests were issued with the node profile rather than the IRSA profile, pointing to an issue with the credentials chain.
After asking on Slack, we thought it might be an issue with IRSA itself (i.e., the pod starting before IRSA credentials were available). In order to examine this possibility, we did two things:
aws sts get-caller-identity
and waiting until the right IAM role comes up: this container always returned the correct IAM role, but some Trino workers were still starting up faulty, so we could confirm that the Pod always received the correct credentialsAfter adding
com.amazonaws=DEBUG
to our logging configuration, we can see how the authentication chain happens in a normal node:After trying static credentials, which don't exist, Trino tries the WebIdentityTokenCredentialsProvider, which works: all good!
(There's a lot of very similar log lines in there, I assume this happens once per process?)
But in a problematic node, we see the following errors:
The credentials chain then tries the next auth steps in the chain, and eventually uses the instance credentials from the node itself, following AWS's default credentials chain.
We managed to workaround the issue for now by setting up a Custom S3 Credentials Provider class, a barebones subclass of com.amazonaws.auth.WebIdentityTokenCredentialsProvider. Since implementing this workaround, we have not seen any auth issues in any of our nodes.
So my bug report is - something is interrupting the WebIdentityTokenCredentialsProvider, causing it to intermittently crash. I don't have enough knowledge to dive into the Trino code, but if someone could I'd be super grateful 🙏
The text was updated successfully, but these errors were encountered: