Skip to content
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

Open
Pluies opened this issue Dec 1, 2022 · 18 comments · May be fixed by #22425
Open

Trino intermittently fails to pick up IRSA #15267

Pluies opened this issue Dec 1, 2022 · 18 comments · May be fixed by #22425

Comments

@Pluies
Copy link
Contributor

Pluies commented Dec 1, 2022

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:

2022-11-08T17:16:20.185Z	ERROR	SplitRunner-821-67	io.trino.execution.executor.TaskExecutor	Error processing Split 20221108_171619_00082_ssjay.1.1.0-12 {path=s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet, start=0, length=9688} (start = 5538933.073355, wall = 11 ms, cpu = 0 ms, wait = 0 ms, calls = 1): HIVE_CANNOT_OPEN_SPLIT: Error opening Hive split s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet (offset=0, length=9688): com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: GWPGEHRBPM0Z41ER; S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg=; Proxy: null), S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg= (Path: s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet)
io.trino.spi.TrinoException: Error opening Hive split s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet (offset=0, length=9688): com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: GWPGEHRBPM0Z41ER; S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg=; Proxy: null), S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg= (Path: s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet)
	at io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createPageSource(ParquetPageSourceFactory.java:290)
	at io.trino.plugin.deltalake.DeltaLakePageSourceProvider.createPageSource(DeltaLakePageSourceProvider.java:199)
	at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:49)
	at io.trino.split.PageSourceManager.createPageSource(PageSourceManager.java:62)
	at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:308)
	at io.trino.operator.Driver.processInternal(Driver.java:411)
	at io.trino.operator.Driver.lambda$process$10(Driver.java:314)
	at io.trino.operator.Driver.tryWithLock(Driver.java:706)
	at io.trino.operator.Driver.process(Driver.java:306)
	at io.trino.operator.Driver.processForDuration(Driver.java:277)
	at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:739)
	at io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:164)
	at io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:515)
	at io.trino.$gen.Trino_402____20221108_171328_2.run(Unknown Source)
	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:833)
Caused by: io.trino.plugin.hive.s3.TrinoS3FileSystem$UnrecoverableS3OperationException: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: GWPGEHRBPM0Z41ER; S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg=; Proxy: null), S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg= (Path: s3a:https://REDACTED/table/0001/block_date=2022-07-09/part-00000-b7d3e0d5-91f0-4a5e-b423-4a115c0dfe47.c000.snappy.parquet)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.lambda$openStream$2(TrinoS3FileSystem.java:1373)
	at io.trino.plugin.hive.util.RetryDriver.run(RetryDriver.java:130)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.openStream(TrinoS3FileSystem.java:1360)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.openStream(TrinoS3FileSystem.java:1345)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.seekStream(TrinoS3FileSystem.java:1338)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.lambda$read$1(TrinoS3FileSystem.java:1282)
	at io.trino.plugin.hive.util.RetryDriver.run(RetryDriver.java:130)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.read(TrinoS3FileSystem.java:1281)
	at java.base/java.io.BufferedInputStream.read1(BufferedInputStream.java:282)
	at java.base/java.io.BufferedInputStream.read(BufferedInputStream.java:343)
	at java.base/java.io.DataInputStream.read(DataInputStream.java:151)
	at java.base/java.io.DataInputStream.read(DataInputStream.java:151)
	at io.trino.hdfs.FSDataInputStreamTail.readTail(FSDataInputStreamTail.java:59)
	at io.trino.filesystem.hdfs.HdfsInput.readTail(HdfsInput.java:56)
	at io.trino.filesystem.TrinoInput.readTail(TrinoInput.java:46)
	at io.trino.plugin.hive.parquet.TrinoParquetDataSource.readTailInternal(TrinoParquetDataSource.java:54)
	at io.trino.parquet.AbstractParquetDataSource.readTail(AbstractParquetDataSource.java:90)
	at io.trino.parquet.reader.MetadataReader.readFooter(MetadataReader.java:98)
	at io.trino.plugin.hive.parquet.ParquetPageSourceFactory.createPageSource(ParquetPageSourceFactory.java:210)
	... 16 more
Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Access Denied (Service: Amazon S3; Status Code: 403; Error Code: AccessDenied; Request ID: GWPGEHRBPM0Z41ER; S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg=; Proxy: null), S3 Extended Request ID: btz5MHY4uuvDZWpIiCeDvsb7Waxfj5+Er8++1ATgMPF5XhcIvraqklFutzc1rH4GBZVwmhoZdfg=
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleErrorResponse(AmazonHttpClient.java:1879)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.handleServiceErrorResponse(AmazonHttpClient.java:1418)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(AmazonHttpClient.java:1387)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper(AmazonHttpClient.java:1157)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:814)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer(AmazonHttpClient.java:781)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute(AmazonHttpClient.java:755)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(AmazonHttpClient.java:715)
	at com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(AmazonHttpClient.java:697)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:561)
	at com.amazonaws.http.AmazonHttpClient.execute(AmazonHttpClient.java:541)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5456)
	at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5403)
	at com.amazonaws.services.s3.AmazonS3Client.getObject(AmazonS3Client.java:1524)
	at io.trino.plugin.hive.s3.TrinoS3FileSystem$TrinoS3InputStream.lambda$openStream$2(TrinoS3FileSystem.java:1365)
	... 34 more

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:

  • Created an initContainer running 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 credentials
  • But maybe it's container-scoped? We added a custom entrypoint to our Trino container to do the same, and, likewise, it always showed the correct IAM role, and yet some worker nodes still started up wrong, so we ruled out this possibility entirely.

After adding com.amazonaws=DEBUG to our logging configuration, we can see how the authentication chain happens in a normal node:

2022-11-18T15:05:47.758Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-6-103	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from EnvironmentVariableCredentialsProvider: Unable to load AWS credentials from environment variables (AWS_ACCESS_KEY_ID (or AWS_ACCESS_KEY) and AWS_SECRET_KEY (or AWS_SECRET_ACCESS_KEY))
2022-11-18T15:05:47.759Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-6-103	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey)
2022-11-18T15:05:47.785Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-58-81	com.amazonaws.request	Sending Request: POST https://sts.eu-west-1.amazonaws.com / (...) 
(...)
2022-11-18T15:05:47.996Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-58-81	com.amazonaws.request	Received successful response: 200, AWS Request ID: 70951960-78a5-4b33-a576-e3473b670e0e
2022-11-18T15:05:47.996Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-58-81	com.amazonaws.requestId	x-amzn-RequestId: 70951960-78a5-4b33-a576-e3473b670e0e
2022-11-18T15:05:47.997Z	DEBUG	20221118_150542_00001_i5juv.1.0.0-58-81	com.amazonaws.auth.AWSCredentialsProviderChain	Loading credentials from WebIdentityTokenCredentialsProvider

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:

2022-11-20T07:46:17.027008338Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-6-70	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026983467Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-2-69	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026962152Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-1-81	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026960176Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-0-71	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026955191Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-7-77	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.02695308Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-3-80	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026950294Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-4-82	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.
2022-11-20T07:46:17.026932937Z stderr F 2022-11-20T07:46:17.026Z	DEBUG	20221120_074616_00009_zjt8k.1.11.0-5-66	com.amazonaws.auth.AWSCredentialsProviderChain	Unable to load credentials from WebIdentityTokenCredentialsProvider: Interrupted waiting to refresh the value.

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 🙏

@skyahead
Copy link

skyahead commented Mar 7, 2023

@Pluies I wonder what is being done in the Custom S3 Credentials Provider that you wrote? Does it keep looking for the web token?

@Pluies
Copy link
Contributor Author

Pluies commented Mar 8, 2023

@skyahead it doesn't even keep looking, it only does it once 😄 Well, actually, being a subclass of AWS' WebIdentityTokenCredentialsProvider, it reuses AWS' logic, which may be retrying; I haven't dug into it.

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 hadoop.xml override to pick it up looks like:

<configuration>
  <property>
    <name>trino.s3.credentials-provider</name>
    <value>com.foo.CustomWebIdentityTokenCredentialsProvider</value>
    <description>Custom IAM credentials provider to force IRSA</description>
  </property>
</configuration>

@skyahead
Copy link

skyahead commented Mar 8, 2023

@skyahead it doesn't even keep looking, it only does it once 😄 Well, actually, being a subclass of AWS' WebIdentityTokenCredentialsProvider, it reuses AWS' logic, which may be retrying; I haven't dug into it.

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 hadoop.xml override to pick it up looks like:

<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

@mccartney
Copy link

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.

I suspect the fact that AWSCredentialsProviderChain has reuseLastProvider defaulting to true doesn't help.

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:

@hashhar
Copy link
Member

hashhar commented Aug 4, 2023

cc: @pettyjamesm Have you by chance ever run into this or have ideas if something is wrong on the Trino side?

@pettyjamesm
Copy link
Member

pettyjamesm commented Aug 4, 2023

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.

@hashhar
Copy link
Member

hashhar commented Aug 5, 2023

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

@osscm
Copy link
Contributor

osscm commented Oct 19, 2023

cc @hashhar @pettyjamesm @mccartney @electrum

We have also faced similar issues and endup implementing a custom credential provider, which internally uses WebIdentityTokenCredentialsProvider and STSAssumeRoleSessionCredentialsProvider.
Though we have to add little a bit more code (picked from the TrinoS3FileSystem ) as we want to support the assume role case as well.
This will pin to the WebIdentityTokenCredentialsProvider and do not fallback to instance profile credentials or any other credentials, which makes its easier to understand as well.

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 CustomCredentialsProvider (based on WebIdentityTokenCredentialsProvider) we have not seen this issue (fingers crossed)
This is being tested/used for good amount of time.

IMO, It would be good to add a WebIdentityTokenCredentialsProvider based credentials provider this in the Trino core code base itself, so that if needed, then based on the config or by overriding the current property trino.s3.credentials-provider user can just use it, instead of implementing again and duplicating the code.

We are happy to contribute this to upstream as well, as looks like it will help other teams as well. thanks!

@kar0t
Copy link

kar0t commented Nov 1, 2023

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:
AWS IAM API Throttling or AWS IAM API Failure + Credential Provider Chaining

We need to fix the credential provider configuration to use WebIdentityTokenCredentialsProvider in few points.

(1) Glue Catalog: we can use hive.metastore.glue.aws-credentials-provider for hive and iceberg catalogs

(2) HDFS-S3 File System: Fix TrinoS3FileSystem.getCustomAWSCredentialsProvider to create WebIdentityTokenCredentialsProvider Instance property otherwise you will get error message like

Caused by: java.lang.NoSuchMethodException: com.amazonaws.auth.WebIdentityTokenCredentialsProvider.(java.net.URI,org.apache.hadoop.conf.Configuration)

(3, Optional) Exchange File System: Add a configuration like exchange.s3.credential-provider and use it in theS3FileSystemExchangeStroage.createAwsCredentialsProvider method to create WebIdentityTokenFileCredentialsProvider

After the trino cluster is deployed, you can check whether credential provider chain is used by setting com.amazonaws.auth.AWSCredentialsProviderChain=DEBUG.

@1ambda
Copy link

1ambda commented Nov 4, 2023

FYI: WebIdentityTokenFileCredentialsProvider is different w/ WebIdentityTokenCredentialsProvider

  • TrinoS3FileSystem uses WebIdentityTokenCredentialsProvider
  • theS3FileSystemExchangeStroage uses WebIdentityTokenFileCredentialsProvider

@1ambda
Copy link

1ambda commented Dec 8, 2023

After the modification, the IRSA Access Denied Issue never happen.

@MehulBatra
Copy link

Hi team, any update on this, did we get this fixed in the later version?

@lmay-r7
Copy link

lmay-r7 commented Jan 19, 2024

Hey, we came across this issue today too, we're on Trino 435. Is this currently being looked into?

@jfmrm
Copy link

jfmrm commented Feb 29, 2024

@1ambda would u be able to provider your config file? It's been quite hard to understand how to piece things together.

@kassett
Copy link

kassett commented Mar 1, 2024

@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 helm get values trino
helm_values.txt

@osscm
Copy link
Contributor

osscm commented Mar 3, 2024 via email

@denniswebb
Copy link

Is the solution simply to update configs to include
hive.metastore.glue.aws-credentials-provider=com.amazonaws.auth.WebIdentityTokenCredentialsProvider where needed if IRSA is all we want to use?

Sorry, I'm a DevOps guy trying to help out the data team and have limited Java and no Trino knowledge.

@findepi
Copy link
Member

findepi commented Jun 19, 2024

For S3 this is done in #22163
For Glue this is being done in #22425

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet