Iceberg provides integration with different AWS services through the iceberg-aws module. This section describes how to use Iceberg with AWS.

Enabling AWS Integration

The iceberg-aws module is bundled with Spark and Flink engine runtimes for all versions from 0.11.0 onwards. However, the AWS clients are not bundled so that you can use the same client version as your application. You will need to provide the AWS v2 SDK because that is what Iceberg depends on. You can choose to use the AWS SDK bundle, or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.

All the default AWS clients use the Apache HTTP Client for HTTP connection management. This dependency is not part of the AWS SDK bundle and needs to be added separately. To choose a different HTTP client library such as URL Connection HTTP Client, see the section client customization for more details.

All the AWS module features can be loaded through custom catalog properties, you can go to the documentations of each engine to see how to load a custom catalog. Here are some examples.

Spark

For example, to use AWS features with Spark 3.4 (with scala 2.12) and AWS clients (which is packaged in the iceberg-aws-bundle), you can start the Spark SQL shell with:

  1. # start Spark SQL client shell
  2. spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.4_2.12:{{ icebergVersion }},org.apache.iceberg:iceberg-aws-bundle:{{ icebergVersion }} \
  3. --conf spark.sql.defaultCatalog=my_catalog \
  4. --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
  5. --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \
  6. --conf spark.sql.catalog.my_catalog.type=glue \
  7. --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO

As you can see, In the shell command, we use --packages to specify the additional iceberg-aws-bundle that contains all relevant AWS dependencies.

To use AWS module with Flink, you can download the necessary dependencies and specify them when starting the Flink SQL client:

  1. # download Iceberg dependency
  2. ICEBERG_VERSION={{ icebergVersion }}
  3. MAVEN_URL=https://repo1.maven.org/maven2
  4. ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg
  5. wget $ICEBERG_MAVEN_URL/iceberg-flink-runtime/$ICEBERG_VERSION/iceberg-flink-runtime-$ICEBERG_VERSION.jar
  6. wget $ICEBERG_MAVEN_URL/iceberg-aws-bundle/$ICEBERG_VERSION/iceberg-aws-bundle-$ICEBERG_VERSION.jar
  7. # start Flink SQL client shell
  8. /path/to/bin/sql-client.sh embedded \
  9. ### Cross-Account and Cross-Region Access
  10. It is a common use case for organizations to have a centralized AWS account for Glue metastore and S3 buckets, and use different AWS accounts and regions for different teams to access those resources.
  11. In this case, a [cross-account IAM role](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_use.html) is needed to access those centralized resources.
  12. Iceberg provides an AWS client factory `AssumeRoleAwsClientFactory` to support this common use case.
  13. This also serves as an example for users who would like to implement their own AWS client factory.
  14. This client factory has the following configurable catalog properties:
  15. | Property | Default | Description |
  16. | --------------------------------- | ---------------------------------------- | ------------------------------------------------------ |
  17. | client.assume-role.arn | null, requires user input | ARN of the role to assume, e.g. arn:aws:iam::123456789:role/myRoleToAssume |
  18. | client.assume-role.region | null, requires user input | All AWS clients except the STS client will use the given region instead of the default region chain |
  19. | client.assume-role.external-id | null | An optional [external ID](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_create_for-user_externalid.html) |
  20. | client.assume-role.timeout-sec | 1 hour | Timeout of each assume role session. At the end of the timeout, a new set of role session credentials will be fetched through an STS client. |
  21. By using this client factory, an STS client is initialized with the default credential and region to assume the specified role.
  22. The Glue, S3 and DynamoDB clients are then initialized with the assume-role credential and region to access resources.
  23. Here is an example to start Spark shell with this client factory:
  24. ```shell
  25. spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.4_2.12:{{ icebergVersion }},org.apache.iceberg:iceberg-aws-bundle:{{ icebergVersion }} \
  26. --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
  27. --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \
  28. --conf spark.sql.catalog.my_catalog.type=glue \
  29. --conf spark.sql.catalog.my_catalog.client.factory=org.apache.iceberg.aws.AssumeRoleAwsClientFactory \
  30. --conf spark.sql.catalog.my_catalog.client.assume-role.arn=arn:aws:iam::123456789:role/myRoleToAssume \
  31. --conf spark.sql.catalog.my_catalog.client.assume-role.region=ap-northeast-1

HTTP Client Configurations

AWS clients support two types of HTTP Client, URL Connection HTTP Client and Apache HTTP Client. By default, AWS clients use Apache HTTP Client to communicate with the service. This HTTP client supports various functionalities and customized settings, such as expect-continue handshake and TCP KeepAlive, at the cost of extra dependency and additional startup latency. In contrast, URL Connection HTTP Client optimizes for minimum dependencies and startup latency but supports less functionality than other implementations.

For more details of configuration, see sections URL Connection HTTP Client Configurations and Apache HTTP Client Configurations.

Configure the following property to set the type of HTTP client:

Property Default Description
http-client.type apache Types of HTTP Client.
urlconnection: URL Connection HTTP Client
apache: Apache HTTP Client

URL Connection HTTP Client Configurations

URL Connection HTTP Client has the following configurable properties:

Property Default Description
http-client.urlconnection.socket-timeout-ms null An optional socket timeout) in milliseconds
http-client.urlconnection.connection-timeout-ms null An optional connection timeout) in milliseconds

Users can use catalog properties to override the defaults. For example, to configure the socket timeout for URL Connection HTTP Client when starting a spark shell, one can add:

  1. --conf spark.sql.catalog.my_catalog.http-client.urlconnection.socket-timeout-ms=80

Apache HTTP Client Configurations

Apache HTTP Client has the following configurable properties:

Property Default Description
http-client.apache.socket-timeout-ms null An optional socket timeout) in milliseconds
http-client.apache.connection-timeout-ms null An optional connection timeout) in milliseconds
http-client.apache.connection-acquisition-timeout-ms null An optional connection acquisition timeout) in milliseconds
http-client.apache.connection-max-idle-time-ms null An optional connection max idle timeout) in milliseconds
http-client.apache.connection-time-to-live-ms null An optional connection time to live) in milliseconds
http-client.apache.expect-continue-enabled null, disabled by default An optional true/false setting that controls whether expect continue) is enabled
http-client.apache.max-connections null An optional max connections) in integer
http-client.apache.tcp-keep-alive-enabled null, disabled by default An optional true/false setting that controls whether tcp keep alive) is enabled
http-client.apache.use-idle-connection-reaper-enabled null, enabled by default An optional true/false setting that controls whether use idle connection reaper) is used

Users can use catalog properties to override the defaults. For example, to configure the max connections for Apache HTTP Client when starting a spark shell, one can add:

  1. --conf spark.sql.catalog.my_catalog.http-client.apache.max-connections=5

Run Iceberg on AWS

Amazon Athena

Amazon Athena provides a serverless query engine that could be used to perform read, write, update and optimization tasks against Iceberg tables. More details could be found here.

Amazon EMR

Amazon EMR can provision clusters with Spark (EMR 6 for Spark 3, EMR 5 for Spark 2), Hive, Flink, Trino that can run Iceberg.

Starting with EMR version 6.5.0, EMR clusters can be configured to have the necessary Apache Iceberg dependencies installed without requiring bootstrap actions. Please refer to the official documentation on how to create a cluster with Iceberg installed.

For versions before 6.5.0, you can use a bootstrap action similar to the following to pre-install all necessary dependencies:

  1. #!/bin/bash
  2. ICEBERG_VERSION={{ icebergVersion }}
  3. MAVEN_URL=https://repo1.maven.org/maven2
  4. ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg
  5. # NOTE: this is just an example shared class path between Spark and Flink,
  6. # please choose a proper class path for production.
  7. LIB_PATH=/usr/share/aws/aws-java-sdk/
  8. ICEBERG_PACKAGES=(
  9. "iceberg-spark-runtime-3.3_2.12"
  10. "iceberg-flink-runtime"
  11. "iceberg-aws-bundle"
  12. )
  13. install_dependencies () {
  14. install_path=$1
  15. download_url=$2
  16. version=$3
  17. shift
  18. pkgs=("$@")
  19. for pkg in "${pkgs[@]}"; do
  20. sudo wget -P $install_path $download_url/$pkg/$version/$pkg-$version.jar
  21. done
  22. }
  23. install_dependencies $LIB_PATH $ICEBERG_MAVEN_URL $ICEBERG_VERSION "${ICEBERG_PACKAGES[@]}"

AWS Glue

AWS Glue provides a serverless data integration service that could be used to perform read, write and update tasks against Iceberg tables. More details could be found here.

AWS EKS

AWS Elastic Kubernetes Service (EKS) can be used to start any Spark, Flink, Hive, Presto or Trino clusters to work with Iceberg. Search the Iceberg blogs page for tutorials around running Iceberg with Docker and Kubernetes.

Amazon Kinesis

Amazon Kinesis Data Analytics provides a platform to run fully managed Apache Flink applications. You can include Iceberg in your application Jar and run it in the platform.