Iceberg provides integration with different AWS services through the iceberg-aws
module. This section describes how to use Iceberg with AWS.
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 URL Connection 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 Apache 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.
For example, to use AWS features with Spark 3.0 and AWS clients version 2.17.131, you can start the Spark SQL shell with:
# add Iceberg dependency ICEBERG_VERSION={{% icebergVersion %}} DEPENDENCIES="org.apache.iceberg:iceberg-spark3-runtime:$ICEBERG_VERSION" # add AWS dependnecy AWS_SDK_VERSION=2.17.131 AWS_MAVEN_GROUP=software.amazon.awssdk AWS_PACKAGES=( "bundle" "url-connection-client" ) for pkg in "${AWS_PACKAGES[@]}"; do DEPENDENCIES+=",$AWS_MAVEN_GROUP:$pkg:$AWS_SDK_VERSION" done # start Spark SQL client shell spark-sql --packages $DEPENDENCIES \ --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \ --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \ --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 AWS bundle and HTTP client dependencies with their version as 2.17.131
.
To use AWS module with Flink, you can download the necessary dependencies and specify them when starting the Flink SQL client:
# download Iceberg dependency ICEBERG_VERSION={{% icebergVersion %}} MAVEN_URL=https://repo1.maven.org/maven2 ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg wget $ICEBERG_MAVEN_URL/iceberg-flink-runtime/$ICEBERG_VERSION/iceberg-flink-runtime-$ICEBERG_VERSION.jar # download AWS dependnecy AWS_SDK_VERSION=2.17.131 AWS_MAVEN_URL=$MAVEN_URL/software/amazon/awssdk AWS_PACKAGES=( "bundle" "url-connection-client" ) for pkg in "${AWS_PACKAGES[@]}"; do wget $AWS_MAVEN_URL/$pkg/$AWS_SDK_VERSION/$pkg-$AWS_SDK_VERSION.jar done # start Flink SQL client shell /path/to/bin/sql-client.sh embedded \ -j iceberg-flink-runtime-$ICEBERG_VERSION.jar \ -j bundle-$AWS_SDK_VERSION.jar \ -j url-connection-client-$AWS_SDK_VERSION.jar \ shell
With those dependencies, you can create a Flink catalog like the following:
CREATE CATALOG my_catalog WITH ( 'type'='iceberg', 'warehouse'='s3://my-bucket/my/key/prefix', 'catalog-impl'='org.apache.iceberg.aws.glue.GlueCatalog', 'io-impl'='org.apache.iceberg.aws.s3.S3FileIO' );
You can also specify the catalog configurations in sql-client-defaults.yaml
to preload it:
catalogs: - name: my_catalog type: iceberg warehouse: s3://my-bucket/my/key/prefix catalog-impl: org.apache.iceberg.aws.glue.GlueCatalog io-impl: org.apache.iceberg.aws.s3.S3FileIO
To use AWS module with Hive, you can download the necessary dependencies similar to the Flink example, and then add them to the Hive classpath or add the jars at runtime in CLI:
add jar /my/path/to/iceberg-hive-runtime.jar; add jar /my/path/to/aws/bundle.jar; add jar /my/path/to/aws/url-connection-client.jar;
With those dependencies, you can register a Glue catalog and create external tables in Hive at runtime in CLI by:
SET iceberg.engine.hive.enabled=true; SET hive.vectorized.execution.enabled=false; SET iceberg.catalog.glue.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog; SET iceberg.catalog.glue.warehouse=s3://my-bucket/my/key/prefix; -- suppose you have an Iceberg table database_a.table_a created by GlueCatalog CREATE EXTERNAL TABLE database_a.table_a STORED BY 'org.apache.iceberg.mr.hive.HiveIcebergStorageHandler' TBLPROPERTIES ('iceberg.catalog'='glue');
You can also preload the catalog by setting the configurations above in hive-site.xml
.
There are multiple different options that users can choose to build an Iceberg catalog with AWS.
Iceberg enables the use of AWS Glue as the Catalog
implementation. When used, an Iceberg namespace is stored as a Glue Database, an Iceberg table is stored as a Glue Table, and every Iceberg table version is stored as a Glue TableVersion. You can start using Glue catalog by specifying the catalog-impl
as org.apache.iceberg.aws.glue.GlueCatalog
, just like what is shown in the enabling AWS integration section above. More details about loading the catalog can be found in individual engine pages, such as Spark and Flink.
There is a unique Glue metastore in each AWS account and each AWS region. By default, GlueCatalog
chooses the Glue metastore to use based on the user's default AWS client credential and region setup. You can specify the Glue catalog ID through glue.id
catalog property to point to a Glue catalog in a different AWS account. The Glue catalog ID is your numeric AWS account ID. If the Glue catalog is in a different region, you should configure you AWS client to point to the correct region, see more details in AWS client customization.
By default, Glue stores all the table versions created and user can rollback a table to any historical version if needed. However, if you are streaming data to Iceberg, this will easily create a lot of Glue table versions. Therefore, it is recommended to turn off the archive feature in Glue by setting glue.skip-archive
to true
. For more details, please read Glue Quotas and the UpdateTable API.
Allow user to skip name validation for table name and namespaces. It is recommended to stick to Glue best practice in https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html to make sure operations are Hive compatible. This is only added for users that have existing conventions using non-standard characters. When database name and table name validation are skipped, there is no guarantee that downstream systems would all support the names.
By default, Iceberg uses Glue's optimistic locking for concurrent updates to a table. With optimistic locking, each table has a version id. If users retrieve the table metadata, Iceberg records the version id of that table. Users can update the table, but only if the version id on the server side has not changed. If there is a version mismatch, it means that someone else has modified the table before you did. The update attempt fails, because you have a stale version of the table. If this happens, Iceberg refreshes the metadata and checks if there might be potential conflict. If there is no commit conflict, the operation will be retried. Optimistic locking guarantees atomic transaction of Iceberg tables in Glue. It also prevents others from accidentally overwriting your changes.
{{< hint info >}} Please use AWS SDK version >= 2.17.131 to leverage Glue's Optimistic Locking. If the AWS SDK version is below 2.17.131, only in-memory lock is used. To ensure atomic transaction, you need to set up a DynamoDb Lock Manager. {{< /hint >}}
Similar to all other catalog implementations, warehouse
is a required catalog property to determine the root path of the data warehouse in storage. By default, Glue only allows a warehouse location in S3 because of the use of S3FileIO
. To store data in a different local or cloud store, Glue catalog can switch to use HadoopFileIO
or any custom FileIO by setting the io-impl
catalog property. Details about this feature can be found in the custom FileIO section.
By default, the root location for a table my_table
of namespace my_ns
is at my-warehouse-location/my-ns.db/my-table
. This default root location can be changed at both namespace and table level.
To use a different path prefix for all tables under a namespace, use AWS console or any AWS Glue client SDK you like to update the locationUri
attribute of the corresponding Glue database. For example, you can update the locationUri
of my_ns
to s3://my-ns-bucket
, then any newly created table will have a default root location under the new prefix. For instance, a new table my_table_2
will have its root location at s3://my-ns-bucket/my_table_2
.
To use a completely different root path for a specific table, set the location
table property to the desired root path value you want. For example, in Spark SQL you can do:
CREATE TABLE my_catalog.my_ns.my_table ( id bigint, data string, category string) USING iceberg OPTIONS ('location'='s3://my-special-table-bucket') PARTITIONED BY (category);
For engines like Spark that supports the LOCATION
keyword, the above SQL statement is equivalent to:
CREATE TABLE my_catalog.my_ns.my_table ( id bigint, data string, category string) USING iceberg LOCATION 's3://my-special-table-bucket' PARTITIONED BY (category);
Iceberg supports using a DynamoDB table to record and manage database and table information.
The DynamoDB catalog supports the following configurations:
Property | Default | Description |
---|---|---|
dynamodb.table-name | iceberg | name of the DynamoDB table used by DynamoDbCatalog |
The DynamoDB table is designed with the following columns:
Column | Key | Type | Description |
---|---|---|---|
identifier | partition key | string | table identifier such as db1.table1 , or string NAMESPACE for namespaces |
namespace | sort key | string | namespace name. A global secondary index (GSI) is created with namespace as partition key, identifier as sort key, no other projected columns |
v | string | row version, used for optimistic locking | |
updated_at | number | timestamp (millis) of the last update | |
created_at | number | timestamp (millis) of the table creation | |
p.<property_key> | string | Iceberg-defined table properties including table_type , metadata_location and previous_metadata_location or namespace properties |
This design has the following benefits:
v
is used instead of updated_at
to avoid 2 processes committing at the same millisecondcatalog.renameTable
to ensure idempotencyowner
, and search tables by owner by adding a GSI on the p.owner
column.Iceberg also supports JDBC catalog which uses a table in a relational database to manage Iceberg tables. You can configure to use JDBC catalog with relational database services like AWS RDS. Read the JDBC integration page for guides and examples about using the JDBC catalog. Read this AWS documentation for more details about configuring JDBC catalog with IAM authentication.
With all the available options, we offer the following guidance when choosing the right catalog to use for your application:
Amazon DynamoDB can be used by HadoopCatalog
or HadoopTables
, so that for every commit, the catalog first obtains a lock using a helper DynamoDB table and then try to safely modify the Iceberg table. This is necessary for a file system-based catalog to ensure atomic transaction in storages like S3 that do not provide file write mutual exclusion.
This feature requires the following lock related catalog properties:
lock-impl
as org.apache.iceberg.aws.dynamodb.DynamoDbLockManager
.lock.table
as the DynamoDB table name you would like to use. If the lock table with the given name does not exist in DynamoDB, a new table is created with billing mode set as pay-per-request.Other lock related catalog properties can also be used to adjust locking behaviors such as heartbeat interval. For more details, please refer to Lock catalog properties.
Iceberg allows users to write data to S3 through S3FileIO
. GlueCatalog
by default uses this FileIO
, and other catalogs can load this FileIO
using the io-impl
catalog property.
S3FileIO
implements a customized progressive multipart upload algorithm to upload data. Data files are uploaded by parts in parallel as soon as each part is ready, and each file part is deleted as soon as its upload process completes. This provides maximized upload speed and minimized local disk usage during uploads. Here are the configurations that users can tune related to this feature:
Property | Default | Description |
---|---|---|
s3.multipart.num-threads | the available number of processors in the system | number of threads to use for uploading parts to S3 (shared across all output streams) |
s3.multipart.part-size-bytes | 32MB | the size of a single part for multipart upload requests |
s3.multipart.threshold | 1.5 | the threshold expressed as a factor times the multipart size at which to switch from uploading using a single put object request to uploading using multipart upload |
s3.staging-dir | java.io.tmpdir property value | the directory to hold temporary files |
S3FileIO
supports all 3 S3 server side encryption modes:
To enable server side encryption, use the following configuration properties:
Property | Default | Description |
---|---|---|
s3.sse.type | none | none , s3 , kms or custom |
s3.sse.key | aws/s3 for kms type, null otherwise | A KMS Key ID or ARN for kms type, or a custom base-64 AES256 symmetric key for custom type. |
s3.sse.md5 | null | If SSE type is custom , this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity. |
S3FileIO
supports S3 access control list (ACL) for detailed access control. User can choose the ACL level by setting the s3.acl
property. For more details, please read S3 ACL Documentation.
S3 and many other cloud storage services throttle requests based on object prefix. Data stored in S3 with a traditional Hive storage layout can face S3 request throttling as objects are stored under the same filepath prefix.
Iceberg by default uses the Hive storage layout, but can be switched to use the ObjectStoreLocationProvider
. With ObjectStoreLocationProvider
, a determenistic hash is generated for each stored file, with the hash appended directly after the write.data.path
. This ensures files written to s3 are equally distributed across multiple prefixes in the S3 bucket. Resulting in minimized throttling and maximized throughput for S3-related IO operations. When using ObjectStoreLocationProvider
having a shared and short write.data.path
across your Iceberg tables will improve performance.
For more information on how S3 scales API QPS, checkout the 2018 re:Invent session on Best Practices for Amazon S3 and Amazon S3 Glacier. At 53:39 it covers how S3 scales/partitions & at 54:50 it discusses the 30-60 minute wait time before new partitions are created.
To use the ObjectStorageLocationProvider
add 'write.object-storage.enabled'=true
in the table's properties. Below is an example Spark SQL command to create a table using the ObjectStorageLocationProvider
:
CREATE TABLE my_catalog.my_ns.my_table ( id bigint, data string, category string) USING iceberg OPTIONS ( 'write.object-storage.enabled'=true, 'write.data.path'='s3://my-table-data-bucket') PARTITIONED BY (category);
We can then insert a single row into this new table
INSERT INTO my_catalog.my_ns.my_table VALUES (1, "Pizza", "orders");
Which will write the data to S3 with a hash (2d3905f8
) appended directly after the write.object-storage.path
, ensuring reads to the table are spread evenly across S3 bucket prefixes, and improving performance.
s3://my-table-data-bucket/2d3905f8/my_ns.db/my_table/category=orders/00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet
Note, the path resolution logic for ObjectStoreLocationProvider
is write.data.path
then <tableLocation>/data
. However, for the older versions up to 0.12.0, the logic is as follows:
write.object-storage.path
must be set.write.object-storage.path
then write.folder-storage.path
then <tableLocation>/data
.For more details, please refer to the LocationProvider Configuration section.
In November 2020, S3 announced strong consistency for all read operations, and Iceberg is updated to fully leverage this feature. There is no redundant consistency wait and check which might negatively impact performance during IO operations.
Before S3FileIO
was introduced, many Iceberg users choose to use HadoopFileIO
to write data to S3 through the S3A FileSystem. As introduced in the previous sections, S3FileIO
adopts latest AWS clients and S3 features for optimized security and performance, and is thus recommend for S3 use cases rather than the S3A FileSystem.
S3FileIO
writes data with s3://
URI scheme, but it is also compatible with schemes written by the S3A FileSystem. This means for any table manifests containing s3a://
or s3n://
file paths, S3FileIO
is still able to read them. This feature allows people to easily switch from S3A to S3FileIO
.
If for any reason you have to use S3A, here are the instructions:
warehouse
catalog property to be an S3A path, e.g. s3a://my-bucket/my-warehouse
HiveCatalog
, to also store metadata using S3A, specify the Hadoop config property hive.metastore.warehouse.dir
to be an S3A path.To ensure integrity of uploaded objects, checksum validations for S3 writes can be turned on by setting catalog property s3.checksum-enabled
to true
. This is turned off by default.
Custom tags can be added to S3 objects while writing and deleting. For example, to write S3 tags with Spark 3.0, you can start the Spark SQL shell with:
spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \ --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.write.tags.my_key1=my_val1 \ --conf spark.sql.catalog.my_catalog.s3.write.tags.my_key2=my_val2
For the above example, the objects in S3 will be saved with tags: my_key1=my_val1
and my_key2=my_val2
. Do note that the specified write tags will be saved only while object creation.
When the catalog property s3.delete-enabled
is set to false
, the objects are not hard-deleted from S3. This is expected to be used in combination with S3 delete tagging, so objects are tagged and removed using S3 lifecycle policy. The property is set to true
by default.
With the s3.delete.tags
config, objects are tagged with the configured key-value pairs before deletion. Users can configure tag-based object lifecycle policy at bucket level to transition objects to different tiers. For example, to add S3 delete tags with Spark 3.0, you can start the Spark SQL shell with:
sh spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.my_catalog.warehouse=s3://iceberg-warehouse/s3-tagging \ --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.delete.tags.my_key3=my_val3 \ --conf spark.sql.catalog.my_catalog.s3.delete-enabled=false
For the above example, the objects in S3 will be saved with tags: my_key3=my_val3
before deletion. Users can also use the catalog property s3.delete.num-threads
to mention the number of threads to be used for adding delete tags to the S3 objects.
For more details on tag restrictions, please refer User-Defined Tag Restrictions.
Access Points can be used to perform S3 operations by specifying a mapping of bucket to access points. This is useful for multi-region access, cross-region access, disaster recovery, etc.
For using cross-region access points, we need to additionally set use-arn-region-enabled
catalog property to true
to enable S3FileIO
to make cross-region calls, it's not required for same / multi-region access points.
For example, to use S3 access-point with Spark 3.0, you can start the Spark SQL shell with:
spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket2/my/key/prefix \ --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ --conf spark.sql.catalog.test.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ --conf spark.sql.catalog.test.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap
For the above example, the objects in S3 on my-bucket1
and my-bucket2
buckets will use arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap
access-point for all S3 operations.
For more details on using access-points, please refer Using access points with compatible Amazon S3 operations.
Many organizations have customized their way of configuring AWS clients with their own credential provider, access proxy, retry strategy, etc. Iceberg allows users to plug in their own implementation of org.apache.iceberg.aws.AwsClientFactory
by setting the client.factory
catalog property.
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. In this case, a cross-account IAM role is needed to access those centralized resources. Iceberg provides an AWS client factory AssumeRoleAwsClientFactory
to support this common use case. This also serves as an example for users who would like to implement their own AWS client factory.
This client factory has the following configurable catalog properties:
Property | Default | Description |
---|---|---|
client.assume-role.arn | null, requires user input | ARN of the role to assume, e.g. arn:aws:iam::123456789:role/myRoleToAssume |
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 |
client.assume-role.external-id | null | An optional external ID |
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 a STS client. |
By using this client factory, an STS client is initialized with the default credential and region to assume the specified role. The Glue, S3 and DynamoDB clients are then initialized with the assume-role credential and region to access resources. Here is an example to start Spark shell with this client factory:
spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:{{% icebergVersion %}},software.amazon.awssdk:bundle:2.17.131 \ --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \ --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \ --conf spark.sql.catalog.my_catalog.client.factory=org.apache.iceberg.aws.AssumeRoleAwsClientFactory \ --conf spark.sql.catalog.my_catalog.client.assume-role.arn=arn:aws:iam::123456789:role/myRoleToAssume \ --conf spark.sql.catalog.my_catalog.client.assume-role.region=ap-northeast-1
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 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:
#!/bin/bash AWS_SDK_VERSION=2.17.131 ICEBERG_VERSION={{% icebergVersion %}} MAVEN_URL=https://repo1.maven.org/maven2 ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg AWS_MAVEN_URL=$MAVEN_URL/software/amazon/awssdk # NOTE: this is just an example shared class path between Spark and Flink, # please choose a proper class path for production. LIB_PATH=/usr/share/aws/aws-java-sdk/ AWS_PACKAGES=( "bundle" "url-connection-client" ) ICEBERG_PACKAGES=( "iceberg-spark3-runtime" "iceberg-flink-runtime" ) install_dependencies () { install_path=$1 download_url=$2 version=$3 shift pkgs=("$@") for pkg in "${pkgs[@]}"; do sudo wget -P $install_path $download_url/$pkg/$version/$pkg-$version.jar done } install_dependencies $LIB_PATH $ICEBERG_MAVEN_URL $ICEBERG_VERSION "${ICEBERG_PACKAGES[@]}" install_dependencies $LIB_PATH $AWS_MAVEN_URL $AWS_SDK_VERSION "${AWS_PACKAGES[@]}"
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 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.