trinodb / trino

Official repository of Trino, the distributed SQL query engine for big data, formerly known as PrestoSQL (https://trino.io)
https://trino.io
Apache License 2.0
10.25k stars 2.95k forks source link

Support remote-signing in Trino Iceberg connector #21189

Open kasun-indrasiri opened 6 months ago

kasun-indrasiri commented 6 months ago

Hi team, I'm trying to use Trino with an external Iceberg REST Catalog implementation. That catalog requires using s3.remote-signing-enabled property when reading Iceberg tables. Therefore, I set that as part of Trino Iceberg catalog properties as follows: iceberg.rest-catalog.s3.remote-signing-enabled=true

However, when I try to start the Trino docker container with that config, I'm getting following error message: Error: Configuration property 'iceberg.rest-catalog.s3.remote-signing-enabled'was not used

Does Trino Iceberg connector support remote signing in Iceberg REST Catalog(https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml) integration?

I was able to use Athena PySpark with the same Iceberg REST Catalog with remote signing enabled.

Here is an excerpt from REST Catalog spec:

AWS Configurations

    The following configurations should be respected when working with tables stored in AWS S3
     - `client.region`: region to configure client for making requests to AWS
     - `s3.access-key-id`: id for for credentials that provide access to the data in S3
     - `s3.secret-access-key`: secret for credentials that provide access to data in S3 
     - `s3.session-token`: if present, this value should be used for as the session token 
     - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification

Thanks, Kasun

findinpath commented 6 months ago

@kasun-indrasiri can you pls disclose which Iceberg REST Catalog implementation are you using?

findinpath commented 6 months ago

Would using iceberg.rest-catalog.vended-credentials-enabled help ? See https://github.com/trinodb/trino/pull/20186

cc @alexjo2144

kasun-indrasiri commented 4 months ago

Yes, I tried with iceberg.rest-catalog.vended-credentials-enabled. No luck.

I'm using Confluent's Tableflow Iceberg REST Catalog. The same use case works with Athena without any issues.

kasun-indrasiri commented 4 months ago

The new error is Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint:

alexjo2144 commented 4 months ago

Do you have a full stack trace? That error can happen in a lot of different places. A full list of the configs you've set would also be helpful.

kasun-indrasiri commented 4 months ago

This is the full stack trace:

com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] at com.amazonaws.auth.AWSCredentialsProviderChain.getCredentials(AWSCredentialsProviderChain.java:142) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.getCredentialsFromContext(AmazonHttpClient.java:1269) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.runBeforeRequestHandlers(AmazonHttpClient.java:845) at com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute(AmazonHttpClient.java:794) 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:5520) at com.amazonaws.services.s3.AmazonS3Client.getBucketRegionViaHeadRequest(AmazonS3Client.java:6501) at com.amazonaws.services.s3.AmazonS3Client.fetchRegionFromCache(AmazonS3Client.java:6473) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5505) at com.amazonaws.services.s3.AmazonS3Client.invoke(AmazonS3Client.java:5467) at com.amazonaws.services.s3.AmazonS3Client.getObjectMetadata(AmazonS3Client.java:1402) at io.trino.hdfs.s3.TrinoS3FileSystem.lambda$getS3ObjectMetadata$7(TrinoS3FileSystem.java:966) at io.trino.hdfs.s3.RetryDriver.run(RetryDriver.java:125) at io.trino.hdfs.s3.TrinoS3FileSystem.getS3ObjectMetadata(TrinoS3FileSystem.java:963) at io.trino.hdfs.s3.TrinoS3FileSystem.getS3ObjectMetadata(TrinoS3FileSystem.java:947) at io.trino.hdfs.s3.TrinoS3FileSystem.getFileStatus(TrinoS3FileSystem.java:509) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:462) at io.trino.filesystem.hdfs.HdfsInputFile.lambda$lazyStatus$2(HdfsInputFile.java:137) at io.trino.hdfs.authentication.NoHdfsAuthentication.doAs(NoHdfsAuthentication.java:25) at io.trino.hdfs.HdfsEnvironment.doAs(HdfsEnvironment.java:134) at io.trino.filesystem.hdfs.HdfsInputFile.lazyStatus(HdfsInputFile.java:137) at io.trino.filesystem.hdfs.HdfsInputFile.length(HdfsInputFile.java:80) at io.trino.filesystem.tracing.Tracing.withTracing(Tracing.java:47) at io.trino.filesystem.tracing.TracingInputFile.length(TracingInputFile.java:79) at io.trino.plugin.iceberg.fileio.ForwardingInputFile.getLength(ForwardingInputFile.java:41) at org.apache.iceberg.avro.AvroIterable.newFileReader(AvroIterable.java:100) at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:76) at org.apache.iceberg.avro.AvroIterable.iterator(AvroIterable.java:36) at org.apache.iceberg.relocated.com.google.common.collect.Iterables.addAll(Iterables.java:333) at org.apache.iceberg.relocated.com.google.common.collect.Lists.newLinkedList(Lists.java:241) at org.apache.iceberg.ManifestLists.read(ManifestLists.java:45) at org.apache.iceberg.BaseSnapshot.cacheManifests(BaseSnapshot.java:146) at org.apache.iceberg.BaseSnapshot.dataManifests(BaseSnapshot.java:172) at org.apache.iceberg.DataTableScan.doPlanFiles(DataTableScan.java:68) at org.apache.iceberg.SnapshotScan.planFiles(SnapshotScan.java:139) at io.trino.plugin.iceberg.IcebergSplitSource.getNextBatch(IcebergSplitSource.java:227) at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource.getNextBatch(ClassLoaderSafeConnectorSplitSource.java:43) at io.trino.split.ConnectorAwareSplitSource.getNextBatch(ConnectorAwareSplitSource.java:73) at io.trino.split.TracingSplitSource.getNextBatch(TracingSplitSource.java:64) at io.trino.split.BufferingSplitSource$GetNextBatch.fetchSplits(BufferingSplitSource.java:130) at io.trino.split.BufferingSplitSource$GetNextBatch.fetchNextBatchAsync(BufferingSplitSource.java:112) at io.trino.split.BufferingSplitSource.getNextBatch(BufferingSplitSource.java:61) at io.trino.split.TracingSplitSource.getNextBatch(TracingSplitSource.java:64) at io.trino.execution.scheduler.SourcePartitionedScheduler.schedule(SourcePartitionedScheduler.java:247) at io.trino.execution.scheduler.SourcePartitionedScheduler$1.schedule(SourcePartitionedScheduler.java:172) at io.trino.execution.scheduler.PipelinedQueryScheduler$DistributedStagesScheduler.schedule(PipelinedQueryScheduler.java:1275) at io.trino.$gen.Trino_439____20240528_183510_2.run(Unknown Source) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572) at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131) at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76) at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82) 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:1583) Suppressed: com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] ... 59 more Suppressed: com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] ... 59 more Suppressed: com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] ... 59 more Suppressed: com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] ... 59 more Suppressed: com.amazonaws.SdkClientException: Unable to load AWS credentials from any provider in the chain: [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)), SystemPropertiesCredentialsProvider: Unable to load AWS credentials from Java system properties (aws.accessKeyId and aws.secretKey), WebIdentityTokenCredentialsProvider: You must specify a value for roleArn and roleSessionName, com.amazonaws.auth.profile.ProfileCredentialsProvider@70fba8de: profile file cannot be null, com.amazonaws.auth.EC2ContainerCredentialsProviderWrapper@5e2eacd0: Failed to connect to service endpoint: ] ... 59 more

alexjo2144 commented 4 months ago

Ah, I think I got the problem. This configuration should fail startup but I didn't remember to check it.

So you need to switch over from the legacy s3 file system to the new native one: https://trino.io/docs/current/object-storage/file-system-s3.html