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

Unable to load credentials from any of the providers in the chain file with native s3 support #23545

Closed
guyco33 opened this issue Sep 24, 2024 · 13 comments · Fixed by #23710
Closed

Comments

@guyco33
Copy link
Member

guyco33 commented Sep 24, 2024

I have an hive external table that points to an s3 location with some CSV gzipped files.
Each 1 minute, one file in the s3 location is updated (replaced) by some external flow.

In 458 lots of queries on this hive external table starts to fail with HIVE_CANNOT_OPEN_SPLIT
Setting back fs.hadoop.enabled=true (as of #23343) fixed the issue.
I tried to workaround the issue by setting s3.retry-mode parameter to STANDARD / ADAPTIVE but with no luck.

I wonder if there is something else that can be done here as it seems that Legacy S3 support will be removed in the future ?

Hive security mapping is enabled with the following hive_security_mapping.json file:

{
  "mappings": [
    {
      "useClusterDefault": true
    }
  ]
}

hive properties for s3 legacy support (no issue)

fs.hadoop.enabled=true
hive.s3.security-mapping.config-file=/usr/lib/trino/etc/hive_security_mapping.json
hive.s3.security-mapping.refresh-period=5s

hive properties for s3 native support

fs.hadoop.enabled=false
fs.native-s3.enabled=true
s3.retry-mode=ADAPTIVE
s3.security-mapping.enabled=true
s3.security-mapping.config-file=/usr/lib/trino/etc/hive_security_mapping.json
s3.security-mapping.refresh-period=5s

Slack discussion here

@guyco33
Copy link
Member Author

guyco33 commented Sep 24, 2024

Here is the stack trace:


io.trino.spi.TrinoException: Error opening Hive split s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/06.gz (offset=0, length=20): Failed to open S3 file: s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/06.gz
	at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:157)
	at io.trino.plugin.hive.HivePageSourceProvider.createHivePageSource(HivePageSourceProvider.java:204)
	at io.trino.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:139)
	at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:48)
	at io.trino.split.PageSourceManager$PageSourceProviderInstance.createPageSource(PageSourceManager.java:79)
	at io.trino.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:261)
	at io.trino.operator.ScanFilterAndProjectOperator$SplitToPages.process(ScanFilterAndProjectOperator.java:192)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:359)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:346)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
	at io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$2(WorkProcessorUtils.java:240)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:261)
	at io.trino.operator.WorkProcessorUtils.lambda$finishWhen$3(WorkProcessorUtils.java:255)
	at io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:423)
	at io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:133)
	at io.trino.operator.Driver.processInternal(Driver.java:403)
	at io.trino.operator.Driver.lambda$process$8(Driver.java:306)
	at io.trino.operator.Driver.tryWithLock(Driver.java:709)
	at io.trino.operator.Driver.process(Driver.java:298)
	at io.trino.operator.Driver.processForDuration(Driver.java:269)
	at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
	at io.trino.execution.executor.timesharing.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:187)
	at io.trino.execution.executor.timesharing.TimeSharingTaskExecutor$TaskRunner.run(TimeSharingTaskExecutor.java:565)
	at io.trino.$gen.Trino_458____20240924_125050_2.run(Unknown Source)
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
	at java.base/java.lang.Thread.run(Thread.java:1570)
Caused by: io.trino.filesystem.TrinoFileSystemException: Failed to open S3 file: s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/06.gz
	at io.trino.filesystem.s3.S3InputStream.seekStream(S3InputStream.java:237)
	at io.trino.filesystem.s3.S3InputStream.read(S3InputStream.java:112)
	at java.base/java.io.InputStream.readNBytes(InputStream.java:412)
	at java.base/java.io.InputStream.readAllBytes(InputStream.java:349)
	at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:141)
	... 31 more
Caused by: software.amazon.awssdk.core.exception.SdkClientException: Unable to load credentials from any of the providers in the chain AwsCredentialsProviderChain(credentialsProviders=[SystemPropertyCredentialsProvider(), EnvironmentVariableCredentialsProvider(), WebIdentityTokenCredentialsProvider(), ProfileCredentialsProvider(profileName=default, profileFile=ProfileFile(sections=[profiles, sso-session], profiles=[Profile(name=default, properties=[region])])), ContainerCredentialsProvider(), InstanceProfileCredentialsProvider()]) : [SystemPropertyCredentialsProvider(): Unable to load credentials from system settings. Access key must be specified either via environment variable (AWS_ACCESS_KEY_ID) or system property (aws.accessKeyId)., EnvironmentVariableCredentialsProvider(): Unable to load credentials from system settings. Access key must be specified either via environment variable (AWS_ACCESS_KEY_ID) or system property (aws.accessKeyId)., WebIdentityTokenCredentialsProvider(): Either the environment variable AWS_WEB_IDENTITY_TOKEN_FILE or the javaproperty aws.webIdentityTokenFile must be set., ProfileCredentialsProvider(profileName=default, profileFile=ProfileFile(sections=[profiles, sso-session], profiles=[Profile(name=default, properties=[region])])): Profile file contained no credentials for profile 'default': ProfileFile(sections=[profiles, sso-session], profiles=[Profile(name=default, properties=[region])]), ContainerCredentialsProvider(): Cannot fetch credentials from container - neither AWS_CONTAINER_CREDENTIALS_FULL_URI or AWS_CONTAINER_CREDENTIALS_RELATIVE_URI environment variables are set., InstanceProfileCredentialsProvider(): Failed to load credentials from IMDS.]
	at software.amazon.awssdk.core.exception.SdkClientException$BuilderImpl.build(SdkClientException.java:111)
	at software.amazon.awssdk.auth.credentials.AwsCredentialsProviderChain.resolveCredentials(AwsCredentialsProviderChain.java:130)
	at software.amazon.awssdk.auth.credentials.internal.LazyAwsCredentialsProvider.resolveCredentials(LazyAwsCredentialsProvider.java:45)
	at software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider.resolveCredentials(DefaultCredentialsProvider.java:129)
	at software.amazon.awssdk.auth.credentials.AwsCredentialsProvider.resolveIdentity(AwsCredentialsProvider.java:54)
	at software.amazon.awssdk.services.s3.auth.scheme.internal.S3AuthSchemeInterceptor.lambda$trySelectAuthScheme$4(S3AuthSchemeInterceptor.java:163)
	at software.amazon.awssdk.core.internal.util.MetricUtils.reportDuration(MetricUtils.java:77)
	at software.amazon.awssdk.services.s3.auth.scheme.internal.S3AuthSchemeInterceptor.trySelectAuthScheme(S3AuthSchemeInterceptor.java:163)
	at software.amazon.awssdk.services.s3.auth.scheme.internal.S3AuthSchemeInterceptor.selectAuthScheme(S3AuthSchemeInterceptor.java:84)
	at software.amazon.awssdk.services.s3.auth.scheme.internal.S3AuthSchemeInterceptor.beforeExecution(S3AuthSchemeInterceptor.java:64)
	at software.amazon.awssdk.core.interceptor.ExecutionInterceptorChain.lambda$beforeExecution$1(ExecutionInterceptorChain.java:59)
	at java.base/java.util.ArrayList.forEach(ArrayList.java:1597)
	at software.amazon.awssdk.core.interceptor.ExecutionInterceptorChain.beforeExecution(ExecutionInterceptorChain.java:59)
	at software.amazon.awssdk.awscore.internal.AwsExecutionContextBuilder.runInitialInterceptors(AwsExecutionContextBuilder.java:242)
	at software.amazon.awssdk.awscore.internal.AwsExecutionContextBuilder.invokeInterceptorsAndCreateExecutionContext(AwsExecutionContextBuilder.java:132)
	at software.amazon.awssdk.awscore.client.handler.AwsSyncClientHandler.invokeInterceptorsAndCreateExecutionContext(AwsSyncClientHandler.java:67)
	at software.amazon.awssdk.core.internal.handler.BaseSyncClientHandler.lambda$execute$0(BaseSyncClientHandler.java:62)
	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:5245)
	at software.amazon.awssdk.services.s3.S3Client.getObject(S3Client.java:9134)
	at io.trino.filesystem.s3.S3InputStream.seekStream(S3InputStream.java:224)
	... 35 more

and thats the error code received with DEBUG mode enabled:

2024-09-24T14:13:35.936Z	DEBUG	SplitRunner-9-74	io.trino.execution.executor.timesharing.TimeSharingTaskExecutor	Error processing Split 20240924_141302_00085_ruxr6.1.0.0-24 HiveSplit{path=s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/24.gz;start=0;length=250;estimatedFileSize=250;hosts=;forceLocalScheduling=false;partitionName=event_name=test_event;deserializerClassName=org.apache.hadoop.hive.serde2.OpenCSVSerde} (start = 1360850.295247, wall = 1023 ms, cpu = 0 ms, wait = 0 ms, calls = 1): HIVE_CANNOT_OPEN_SPLIT: Error opening Hive split s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/24.gz (offset=0, length=250): Failed to open S3 file: s3a://test-bucket-165354/LIVE_PART_EVENTS/last_hour/event_name=test_event/24.gz

@guyco33
Copy link
Member Author

guyco33 commented Sep 24, 2024

It's important to note that the failed query succeed when running it again shortly after, so it's not about s3 access but more about retries

@nineinchnick
Copy link
Member

nineinchnick commented Sep 24, 2024

The root cause might be the same as in #15267. I don't think we should retry on these kind of errors, as we can't tell if they're intermediate (caused by some network or IO issue) or permanent (invalid credentials).

How did you set up S3 authentication? In native FS, it's not possible to use a specific credential provider in the AWS SDK, instead of relying on the default chain of providers, except for using the WebIdentityTokenCredentialProvider, by setting s3.use-web-identity-token-credentials-provider.

@guyco33
Copy link
Member Author

guyco33 commented Sep 25, 2024

I'm running Trino on EC2 and the S3 authentication is done through IAM role that has specific policy allows access to s3 bucket (read & write)
All s3 access and authentication is relying on the default chain of providers that is the AWS IAM role

@nineinchnick
Copy link
Member

@electrum can you advise what can we do in this situation?

@guyco33
Copy link
Member Author

guyco33 commented Sep 25, 2024

I found that the issue is not related to reading csv files while they are being updated (overwritten), but it can happen sporadically for any s3 file.
It seems that for unknown reason the AWS SDK is losing the credentials.

@guyco33
Copy link
Member Author

guyco33 commented Sep 26, 2024

UPDATE: Issue happens only when hive_security_mapping.json is enabled with the native s3.

hive_security_mapping.json:

{
  "mappings": [
    {
      "useClusterDefault": true
    }
  ]
}

setting in hive.properties

s3.security-mapping.enabled=true
s3.security-mapping.config-file=/usr/lib/trino/etc/hive_security_mapping.json
s3.security-mapping.refresh-period=5s

After disabling the hive security mapping with the native s3, issue doesn't happen.

With the Legacy S3 support (fs.hadoop.enabled=true) issued doesn't happen also when security mapping is enabled.

@electrum
Copy link
Member

electrum commented Sep 26, 2024

Which credential provider should be used for your setup?

Does the problem go away if you set the following?

s3.use-web-identity-token-credentials-provider=true

@guyco33
Copy link
Member Author

guyco33 commented Sep 29, 2024

After setting s3.region there are no performance issues.
It seems that the region needs to be set explicitly with the native s3 support to make the i/o latency stable.

Also I had to remove the security mapping file to avoid the errors of Unable to load credentials from any of the providers in the chain

@guyco33
Copy link
Member Author

guyco33 commented Oct 6, 2024

It seems that while running cluster with native s3 support some queries become much more slower.

I succeeded to reproduce the issue with 2 clusters: A (legacy support) and B (native support)
Attached screen shots with queries stats (each line is for stats of one query)
Some queries of B gets much higher wall time although nothing was changes with the underlined s3 data files.

image (1)
image (2)

@anusudarsan
Copy link
Member

@guyco33 can we keep the performance issues in a different issue or on slack? it looks like we are discussing multiple issues here. This issue can be just focussed on how the security mapping file is causing issues intermittently Unable to load credentials from any of the providers in the chain

@nineinchnick
Copy link
Member

nineinchnick commented Oct 7, 2024

@electrum with S3 security mapping enabled, we create a new S3 client for every query. This increases the chance of hitting issues with setting up IAM role authentication. WDYT about adding some LRU client cache per location?

I'm not sure how this works in HDFS, I'm looking it up now.

@electrum
Copy link
Member

electrum commented Oct 7, 2024

@nineinchnick That sound like a good idea.

@guyco33 guyco33 changed the title Hive external table fails to open an updated gzip csv file with native s3 support Unable to load credentials from any of the providers in the chain file with native s3 support Oct 9, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Development

Successfully merging a pull request may close this issue.

4 participants