You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/12/08 19:00:12 UTC

[GitHub] [iceberg] jackye1995 opened a new pull request #1891: AWS: documentation page for AWS module

jackye1995 opened a new pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891


   Initial draft for documentation page of the AWS module, I will mark this as draft until #1823 and #1844 are merged.
   
   Also, I am currently placing it under `Tables` tab, which is clearly wrong. I was thinking about adding a new tab `Integrations` for all integrations including AWS and Nessie, but the navbar becomes too long. I am thinking about moving Flink, Hive and Presto all to a single `Engines` tab, any thoughts on that?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554256044



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider

Review comment:
       Yes that's better, let me also remove other class names in titles

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS ('location'='s3://my-special-table-bucket')

Review comment:
       oh yeah I completely forgot that

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switched to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+Here is an example Spark SQL command to create a table with this feature enabled:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS (
+    'write.object-storage.enabled'=true, 
+    'write.object-storage.path'='s3://my-table-data-bucket')
+PARTITIONED BY (category);
+```
+
+For more details, please refer to the [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/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.
+
+### Hadoop S3A FileSystem
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+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:
+
+1. To store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to also store metadata using S3A, specify the Hadoop config property `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency of your compute engine.
+4. Configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, S3A configuration varies a lot based on the version you use).   
+
+## AWS client customization
+
+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.
+
+### AssumeRoleAwsClientFactory

Review comment:
       Yes I just updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#issuecomment-756489628


   @jackye1995, we can save some space by getting rid of the next/previous links and by moving "About" into "Project":
   
   ```diff
   diff --git a/site/docs/css/extra.css b/site/docs/css/extra.css
   index 4fa7c8036..3d79de02b 100644
   --- a/site/docs/css/extra.css
   +++ b/site/docs/css/extra.css
   @@ -28,6 +28,10 @@
      float: left;
    }
    
   +.navbar-right {
   +  display: none;
   +}
   +
    .navbar-brand {
      margin-right: 1em;
    }
   diff --git a/site/mkdocs.yml b/site/mkdocs.yml
   index 1ce05135b..209495896 100644
   --- a/site/mkdocs.yml
   +++ b/site/mkdocs.yml
   @@ -40,8 +40,8 @@ markdown_extensions:
      - admonition
      - pymdownx.tilde
    nav:
   -  - About: index.md
   -  - Project:
   +  - About:
   +    - Project: index.md
        - Community: community.md
        - Releases: releases.md
        - Trademarks: trademarks.md
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553687370



##########
File path: site/docs/configuration.md
##########
@@ -35,6 +35,20 @@ The properties can be manually constructed or passed in from a compute engine li
 Spark uses its session properties as catalog properties, see more details in the [Spark configuration](#spark-configuration) section.
 Flink passes in catalog properties through `CREATE CATALOG` statement, see more details in the [Flink](../flink/#creating-catalogs-and-using-catalogs) section.
 
+### Lock catalog properties
+
+Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.
+
+| Property                          | Default            | Description                                            |
+| --------------------------------- | ------------------ | ------------------------------------------------------ |
+| lock.impl                         | null               | a custom implementation of the lock manager, the actual interface depends on the catalog used  |
+| lock.table                        | null               | an optional auxiliary table for locking                |

Review comment:
       You may also want to link to the Dynamo docs if you haven't already, since that's the only implementation currently available.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ismailsimsek commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
ismailsimsek commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r544238698



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS

Review comment:
       Thank you @jackye1995  it looks great, this is more an question. is it also possible to run iceberg with AWS "Glue Job"? if we use `spark.jars.packages` to provide iceberg library.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] HeartSaVioR commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r544770511



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO

Review comment:
       I was about to ask the rationalization of S3 FileIO compared to Hadoop filesystem API with S3 support in #1945, but this section covers it. Thanks!
   
   Probably worth to also mention whether Hadoop FS API with S3 is sufficient to work with, or S3 FileIO is required to avoid consistency glitches. That would help end users to determine whether including aws module is a kind of requirement for dealing with S3 or not.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547554300



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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 configuration](#aws-client-configurations).
+
+### Skip Archive
+
+By default, Glue will store 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 `gluecatalog.skip-archive` to false.

Review comment:
       I think it would be a bit cleaner if these catalog options were shorter. this could simply be `skip-archive` in the catalog config. Similarly, the one above could be `catalog-id`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547553849



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID

Review comment:
       What about adding a table of configuration options instead of sections?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547552976



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 

Review comment:
       Is this true? I thought that only one version of the table was kept and that it pointed to the Iceberg root metadata file. That file contains more than one snapshot.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547552679



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:

Review comment:
       You may want to call out the part that matters here, that this is adding `software.amazon.awssdk:bundle:2.15.40` to the `--packages` argument.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547085074



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO

Review comment:
       I have updated strong consistency section for more details, please let me know if it is enough.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554256535



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switched to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+Here is an example Spark SQL command to create a table with this feature enabled:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS (
+    'write.object-storage.enabled'=true, 
+    'write.object-storage.path'='s3://my-table-data-bucket')
+PARTITIONED BY (category);
+```
+
+For more details, please refer to the [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/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.
+
+### Hadoop S3A FileSystem
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+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:
+
+1. To store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to also store metadata using S3A, specify the Hadoop config property `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency of your compute engine.
+4. Configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, S3A configuration varies a lot based on the version you use).   
+
+## AWS client customization
+
+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.
+
+### AssumeRoleAwsClientFactory

Review comment:
       Is there a better heading for this as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547552482



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.

Review comment:
       How about "You will need to provide the AWS v2 SDK because that is what Iceberg depends on." Saying "instead of v1" makes it sound like you can't use both, so I would omit that as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547049733



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)

Review comment:
       Fixed




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553673788



##########
File path: site/docs/configuration.md
##########
@@ -35,6 +35,20 @@ The properties can be manually constructed or passed in from a compute engine li
 Spark uses its session properties as catalog properties, see more details in the [Spark configuration](#spark-configuration) section.
 Flink passes in catalog properties through `CREATE CATALOG` statement, see more details in the [Flink](../flink/#creating-catalogs-and-using-catalogs) section.
 
+### Lock catalog properties
+
+Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.
+
+| Property                          | Default            | Description                                            |
+| --------------------------------- | ------------------ | ------------------------------------------------------ |
+| lock.impl                         | null               | a custom implementation of the lock manager, the actual interface depends on the catalog used  |

Review comment:
       The other implementation class properties are `something-impl`. Shouldn't this be `lock-impl` instead to match those? Then the other properties are in the `lock` namespace.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547077119



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)

Review comment:
       default to `Runtime.getRuntime().availableProcessors()`, let me add in the doc




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553494011



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 

Review comment:
       Snapshot is probably a bad word, I will just use table version then. 
   But Glue also stores all historical table versions for each update.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547065365



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS
+
+[Amazon EMR](https://aws.amazon.com/emr/) is the most common platform to run Iceberg on AWS. EMR can provision clusters with [Spark](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark.html) (EMR 6 for Spark 3, EMR 5 for Spark 2),

Review comment:
       The v2 library is not bundled in spark.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547555029



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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 configuration](#aws-client-configurations).
+
+### Skip Archive
+
+By default, Glue will store 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 `gluecatalog.skip-archive` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, defaults to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, defaults to 20 minutes

Review comment:
       What about `lock-wait-ms` and `lock-expire-ms`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#issuecomment-756448636


   @rdblue there are too many tabs currently on Iceberg website, and adding the additional tab for AWS will make the UI ugly. So I moved Flink, Trino and Hive all under an engine's tab, and added a new tab called Integrations that will have topics like AWS, Nessie and JDBC.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554255438



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider

Review comment:
       How about using a description here rather than a class name, like "Object store file layout" or something?

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS ('location'='s3://my-special-table-bucket')

Review comment:
       You can also use the `LOCATION` DDL clause

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switched to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+Here is an example Spark SQL command to create a table with this feature enabled:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS (
+    'write.object-storage.enabled'=true, 
+    'write.object-storage.path'='s3://my-table-data-bucket')
+PARTITIONED BY (category);
+```
+
+For more details, please refer to the [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/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.
+
+### Hadoop S3A FileSystem
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+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:
+
+1. To store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to also store metadata using S3A, specify the Hadoop config property `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency of your compute engine.
+4. Configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, S3A configuration varies a lot based on the version you use).   
+
+## AWS client customization
+
+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.
+
+### AssumeRoleAwsClientFactory

Review comment:
       Is there a better heading for this as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r545317399



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS

Review comment:
       I have not tried that yet since I am mostly focused on Athena use case, but let me check if anyone has done that, thanks for bringing up this use case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547065239



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.

Review comment:
       Yes that is the default at service side.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] massdosage commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r544982242



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user to write data to S3 through `S3FileIO`.

Review comment:
       ```suggestion
   Iceberg allows users to write data to S3 through `S3FileIO`.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)

Review comment:
       Is there a default here? Or is there no default and setting this is required?

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.

Review comment:
       ```suggestion
   Iceberg by default uses the Hive storage layout, but can be configured to use a different `ObjectStoreLocationProvider`.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.

Review comment:
       ```suggestion
   When creating a new output file using `OutputFile.create()`, a strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 

Review comment:
       ```suggestion
   Therefore, we open a configurer interface for Iceberg users to plug in any client configuration in a centralized place. 
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS
+
+[Amazon EMR](https://aws.amazon.com/emr/) is the most common platform to run Iceberg on AWS. EMR can provision clusters with [Spark](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark.html) (EMR 6 for Spark 3, EMR 5 for Spark 2),
+[Hive](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-hive.html), [Flink](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html),
+[Presto](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-presto.html) that can run Iceberg.
+
+[Amazon Kinesis Data Analytics](https://aws.amazon.com/about-aws/whats-new/2019/11/you-can-now-run-fully-managed-apache-flink-applications-with-apache-kafka/) also provdies a way 
+to run fully managed Apache Flink application. You can include Iceberg in your application Jar and run it in the platform.

Review comment:
       ```suggestion
   to run fully managed Apache Flink applications. You can include Iceberg in your application Jar and run it in the platform.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB

Review comment:
       ```suggestion
   * `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, defaults to 32MB
   ```
   or
   ```suggestion
   * `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default is 32MB
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5

Review comment:
       as above `defaults to` or `default is`

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`

Review comment:
       As above for defaults.

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value

Review comment:
       as above to/is

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  

Review comment:
       ```suggestion
   For more details, please refer to the [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to see how to enable this feature.  
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A

Review comment:
       ```suggestion
   ### Hadoop S3A FileSystem
   ```
   I think it's useful to refer to the FileSystem otherwise some people might get confused by the differences between S3, S3A, S3N etc.

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 

Review comment:
       ```suggestion
   1. To store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   

Review comment:
       ```suggestion
   3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version as S3A configuration varies a lot based on the version you use) .
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.

Review comment:
       ```suggestion
   2. For `HiveCatalog`, to also store metadata using S3A, specify the Hadoop config property `hive.metastore.warehouse.dir` to be an S3A path.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),

Review comment:
       ```suggestion
   When this configurer is used, a STS client is initialized with a default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.

Review comment:
       ```suggestion
    and is thus recommend for S3 use cases rather than the S3A FileSystem.
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS
+
+[Amazon EMR](https://aws.amazon.com/emr/) is the most common platform to run Iceberg on AWS. EMR can provision clusters with [Spark](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark.html) (EMR 6 for Spark 3, EMR 5 for Spark 2),
+[Hive](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-hive.html), [Flink](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-flink.html),
+[Presto](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-presto.html) that can run Iceberg.
+
+[Amazon Kinesis Data Analytics](https://aws.amazon.com/about-aws/whats-new/2019/11/you-can-now-run-fully-managed-apache-flink-applications-with-apache-kafka/) also provdies a way 

Review comment:
       ```suggestion
   [Amazon Kinesis Data Analytics](https://aws.amazon.com/about-aws/whats-new/2019/11/you-can-now-run-fully-managed-apache-flink-applications-with-apache-kafka/) also provides a way 
   ```

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:

Review comment:
       Here you are talking about storing data using `s3a://` URLs as opposed to the above discussion about the S3A FileSystem right? This should probably be reworded to be clear about that if that is the case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547553701



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).

Review comment:
       I would definitely give a full example of using `GlueCatalog` here, in addition to linking to Spark and Flink pages.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547553513



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),

Review comment:
       I would drop references to Iceberg classes here. Most readers are going to care about tables through a SQL or DataFrame interface, not through Iceberg's API. The mapping to Glue database and table names is probably pretty clear.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554256044



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider

Review comment:
       Yes that's better, let me also remove other class names in titles




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553675319



##########
File path: site/docs/configuration.md
##########
@@ -35,6 +35,20 @@ The properties can be manually constructed or passed in from a compute engine li
 Spark uses its session properties as catalog properties, see more details in the [Spark configuration](#spark-configuration) section.
 Flink passes in catalog properties through `CREATE CATALOG` statement, see more details in the [Flink](../flink/#creating-catalogs-and-using-catalogs) section.
 
+### Lock catalog properties
+
+Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.
+
+| Property                          | Default            | Description                                            |
+| --------------------------------- | ------------------ | ------------------------------------------------------ |
+| lock.impl                         | null               | a custom implementation of the lock manager, the actual interface depends on the catalog used  |
+| lock.table                        | null               | an optional auxiliary table for locking                |

Review comment:
       Yes, this is not a part of AWS documentation, that is why I say it as optional, I can remove that. I will emphasize it is required in the AWS doc.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553673960



##########
File path: site/docs/configuration.md
##########
@@ -35,6 +35,20 @@ The properties can be manually constructed or passed in from a compute engine li
 Spark uses its session properties as catalog properties, see more details in the [Spark configuration](#spark-configuration) section.
 Flink passes in catalog properties through `CREATE CATALOG` statement, see more details in the [Flink](../flink/#creating-catalogs-and-using-catalogs) section.
 
+### Lock catalog properties
+
+Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.
+
+| Property                          | Default            | Description                                            |
+| --------------------------------- | ------------------ | ------------------------------------------------------ |
+| lock.impl                         | null               | a custom implementation of the lock manager, the actual interface depends on the catalog used  |
+| lock.table                        | null               | an optional auxiliary table for locking                |

Review comment:
       Isn't the table required if you're using Dynamo? I would not say it is optional, although it is optional if you're not using Dynamo.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547557073



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).

Review comment:
       Sorry, there's one above. Maybe just point the reader to it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547065129



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).

Review comment:
       I see, please read if the new version is easier to understand.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553495410



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID

Review comment:
       The reason I am doing this is because the explanation is quite long, which makes the table look very messy.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547556329



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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 configuration](#aws-client-configurations).
+
+### Skip Archive
+
+By default, Glue will store 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 `gluecatalog.skip-archive` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, defaults to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, defaults to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared across all output streams), defaults to the available number of processors in the system
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, defaults to 32MB
+* `s3fileio.multipart.threshold`: 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, defaults to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, defaults to Java's `java.io.tmpdir` property value

Review comment:
       Great info here, but I think it may be easier to maintain as a table.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553674606



##########
File path: site/mkdocs.yml
##########
@@ -54,14 +54,16 @@ nav:
     - Maintenance: maintenance.md
     - Performance: performance.md
     - Reliability: reliability.md
-  - Spark:
-    - Getting Started: getting-started.md
+
+  - Engines:

Review comment:
       The Spark page is getting long enough that I need to break it into separate pages. That's why I reorganized. I like having an `Integrations` tab, though.
   
   I wonder if we can get rid of forward/back instead. And maybe move "About" into the "Project" list.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554255815



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS ('location'='s3://my-special-table-bucket')

Review comment:
       You can also use the `LOCATION` DDL clause




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553672637



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID

Review comment:
       I think it makes sense here, but I would opt for tables in most other places. Part of the appeal is it forces you to be concise.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553676871



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,187 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock.impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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 [runtime packages](#runtime-packages) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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](#aws-client-customization).
+
+### Skip Archive
+
+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 `gluecatalog.skip-archive` to true.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+To enable this feature, use the lock related catalog properties:
+
+1. Set `lock.impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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 through the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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 root location can be changed at both namespace and table level.
+
+To use a different path prefix for all tables under a namespace, use 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 example, 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 be the desired root path value.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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                                            |
+| --------------------------------- | -------------------------------------------------- | ------------------------------------------------------ |
+| s3fileio.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)  |
+| s3fileio.multipart.part.size      | 32MB                                               | the size of a single part for multipart upload requests  |
+| s3fileio.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  |
+| s3fileio.staging.dir              | `java.io.tmpdir` property value                    | the directory to hold temporary files  |

Review comment:
       Looks like there are multiple discussions around the config key names, and these names for `s3fileio` are not designed by me but Daniel. Let me put up another thread for this discussion before release.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#issuecomment-757054891


   > @jackye1995, we can save some space by getting rid of the next/previous links and by moving "About" into "Project":
   
   Thanks for the code reference, I just updated the tabs based on that


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553650454



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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 configuration](#aws-client-configurations).
+
+### Skip Archive
+
+By default, Glue will store 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 `gluecatalog.skip-archive` to false.

Review comment:
       This is mostly trying to be consistent with the `s3fileio` properties by having `gluecatalog` as the config namespace. We might be able to make all of them shorter, something like `glue.id` and `s3.staging.dir`. Any thoughts on this? @danielcweeks 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#issuecomment-749836081


   @jackye1995 this is great! I noted a few things, but mostly what I would change is organizing the configuration into tables. The text describing all of the features is clear and really helpful.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547084682



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:

Review comment:
       Thank you, I updated the section, please let me know if it is clearer.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547046975



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.

Review comment:
       Yes




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553673270



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,187 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock.impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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 [runtime packages](#runtime-packages) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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](#aws-client-customization).
+
+### Skip Archive
+
+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 `gluecatalog.skip-archive` to true.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+To enable this feature, use the lock related catalog properties:
+
+1. Set `lock.impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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 through the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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 root location can be changed at both namespace and table level.
+
+To use a different path prefix for all tables under a namespace, use 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 example, 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 be the desired root path value.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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                                            |
+| --------------------------------- | -------------------------------------------------- | ------------------------------------------------------ |
+| s3fileio.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)  |
+| s3fileio.multipart.part.size      | 32MB                                               | the size of a single part for multipart upload requests  |

Review comment:
       Have we released this yet? We would normally make it `s3fileio.multipart.part-size-bytes`:
   * `part` is part of "part size" and isn't really part of the hierarchy
   * We prefer to have clear units




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#issuecomment-757054891


   > @jackye1995, we can save some space by getting rid of the next/previous links and by moving "About" into "Project":
   
   Thanks for the code reference, I just updated the tabs based on that


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554257774



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS ('location'='s3://my-special-table-bucket')

Review comment:
       oh yeah I completely forgot that




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553729477



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,187 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock.impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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 [runtime packages](#runtime-packages) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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](#aws-client-customization).
+
+### Skip Archive
+
+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 `gluecatalog.skip-archive` to true.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+To enable this feature, use the lock related catalog properties:
+
+1. Set `lock.impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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 through the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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 root location can be changed at both namespace and table level.
+
+To use a different path prefix for all tables under a namespace, use 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 example, 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 be the desired root path value.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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                                            |
+| --------------------------------- | -------------------------------------------------- | ------------------------------------------------------ |
+| s3fileio.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)  |
+| s3fileio.multipart.part.size      | 32MB                                               | the size of a single part for multipart upload requests  |
+| s3fileio.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  |
+| s3fileio.staging.dir              | `java.io.tmpdir` property value                    | the directory to hold temporary files  |

Review comment:
       let's use #2050 for the discussion




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553649893



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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 configuration](#aws-client-configurations).
+
+### Skip Archive
+
+By default, Glue will store 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 `gluecatalog.skip-archive` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, defaults to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, defaults to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared across all output streams), defaults to the available number of processors in the system
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, defaults to 32MB
+* `s3fileio.multipart.threshold`: 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, defaults to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, defaults to Java's `java.io.tmpdir` property value

Review comment:
       switched to table layout.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554255438



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider

Review comment:
       How about using a description here rather than a class name, like "Object store file layout" or something?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r554258395



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,244 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+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.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the Spark SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock-impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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](#enabling-aws-integration) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+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](#aws-client-customization).
+
+### Skip Archive
+
+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](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, [DynamoDB](https://aws.amazon.com/dynamodb) can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+This feature requires the following lock related catalog properties:
+
+1. Set `lock-impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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:
+
+```sql
+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);
+```
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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  |
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+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. |
+
+### S3 Access Control List
+
+`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](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switched to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+Here is an example Spark SQL command to create a table with this feature enabled:
+
+```sql
+CREATE TABLE my_catalog.my_ns.my_table (
+    id bigint,
+    data string,
+    category string)
+USING iceberg
+OPTIONS (
+    'write.object-storage.enabled'=true, 
+    'write.object-storage.path'='s3://my-table-data-bucket')
+PARTITIONED BY (category);
+```
+
+For more details, please refer to the [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/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.
+
+### Hadoop S3A FileSystem
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+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:
+
+1. To store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to also store metadata using S3A, specify the Hadoop config property `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency of your compute engine.
+4. Configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, S3A configuration varies a lot based on the version you use).   
+
+## AWS client customization
+
+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.
+
+### AssumeRoleAwsClientFactory

Review comment:
       Yes I just updated




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r547557168



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \

Review comment:
       Does the Glue catalog support custom database locations?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] yyanyy commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
yyanyy commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r544709588



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).

Review comment:
       Nit: I guess the logic here isn't super clear to me or I misunderstood; it seems like the "this is because" part is not explaining why we want to use aws account ID, but rather to explain we can configure region and other stuff when we need to? 

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.

Review comment:
       From the current [code base](https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java#L162) it seems like we are not defaulting it to anything, is this `aws/s3` a default value on AWS SDK or something? 

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)

Review comment:
       since we mention minimum dependency here, should we also mention the minimum list of "individual AWS client packages" here or maybe refer to the build file so that people know what packages have to be included if they want to do that?

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.

Review comment:
       Guess this might need to be updated as it seems that we do want to include `iceberg-aws` now, just without AWS dependencies? 

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.

Review comment:
       For things like setting glue catalog id we probably want to mention the parameter "gluecatalog.id"; or overall I think we may want to add a section in the configuration page that include all configurable parameters mentioned in `AwsProperties` and link to that section in this page, otherwise people won't know how to configure them. And that would serve as a centralized place to read about all configurations. Is it part of the plan/covered by other PRs? 

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+
+### DynamoDB for locking Glue tables
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB lock is enabled by default for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+User must specify a table name through catalog property `gluecatalog.lock.table` as the helper DynamoDB lock table to use.
+It is recommend to use the same DynamoDB table for operations in the same Glue catalog,
+and use a different table for a different Glue catalog in another account or region.
+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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+The lock has the following additional properties:
+
+* `gluecatalog.lock.wait-ms`:  max time to wait for lock acquisition, default to 3 minutes
+* `gluecatalog.lock.expire-ms`: max time a table can be locked by a process, default to 20 minutes
+
+If your use case only consists of single-process low-frequency (e.g. hourly, daily) updates to a table,
+you can also turn off this locking feature by setting `gluecatalog.lock.enabled` as false.
+
+### Warehouse Location
+
+By default, Glue uses `S3FileIO` and only allows a warehouse location in S3. 
+To store data in a different local or cloud store, Glue catalog can switch to use `HadoopFileIO` 
+or any custom FileIO using the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+## S3 FileIO
+
+Iceberg allows user 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.
+
+### Progressive Multipart Upload
+
+`S3FileIO` implements a customized progressive multipart upload algorithm to upload data.
+Data files are uploaded in 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 user can tune related to this feature:
+
+* `s3fileio.multipart.num-threads`: number of threads to use for uploading parts to S3 (shared pool across all output streams)
+* `s3fileio.multipart.part.size`: the size of a single part for multipart upload requests, default to 32MB
+* `s3fileio.multipart.threshold`: 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, default to 1.5
+* `s3fileio.staging.dir`: the directory to hold temporary files, default to Java's `java.io.tmpdir` property value
+
+### S3 Server Side Encryption
+
+`S3FileIO` supports all 3 S3 server side encryption modes:
+
+* [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data.
+* [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region.
+* [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption, when you access your objects.
+
+To enable server side encryption, use the following configuration properties:
+
+* `s3fileio.sse.type`: `none`, `s3`, `kms` or `custom`, default to `none`
+* `s3fileio.sse.key`: a KMS Key ID or ARN for `kms` type (default to `aws/s3`), or a custom base-64 AES256 symmetric key for `custom` type.
+* `s3fileio.sse.md5`: if SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity.
+
+### S3 Access Control List
+
+`S3FileIO` supports S3 access control list for detailed access control. 
+User can choose the ACL level by setting the `s3fileio.acl` property.
+For more details, please read [S3 ACL Documentation](https://docs.aws.amazon.com/AmazonS3/latest/dev/acl-overview.html).
+
+### ObjectStoreLocationProvider
+
+S3 and many other cloud storage services [throttle requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/). 
+This means data stored in a traditional Hive storage layout has bad read and write throughput since data files of the same partition are placed under the same prefix.
+Iceberg by default uses the Hive storage layout, but can be switch to use a different `ObjectStoreLocationProvider`.
+In this mode, a hash string is added to the beginning of each file path, so that files are equally distributed across all prefixes in an S3 bucket.
+This results in minimized throttling and maximized throughput for S3-related IO operations.
+For more details, please follow [LocationProvider Configuration](../custom-catalog/#custom-location-provider-implementation) section to enable this feature.  
+
+### S3 Strong Consistency
+
+In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all GET and LIST operations, and Iceberg is updated to fully leverage this feature.
+When creating a new output file using `OutputFile.create()`, strong consistency check is used and an `AlreadyExistsException` will be thrown if the file already exists in S3.
+
+### Hadoop S3A
+
+Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java).
+As introduced in the above sections, the `S3FileIO` adopts latest AWS clients and S3 features for optimzied security and performance,
+ and is thus recommend for S3 use case over S3A.
+
+`S3FileIO` is compatible with legacy URI schemes written by S3A, 
+so any existing tables with `s3a://` or `s3n://` file paths are treated as equivalent `s3://` file paths.    
+
+If for any reason you have to use S3A, here are the instructions:
+
+1. to store data using S3A, specify the `warehouse` catalog property to be an S3A path, e.g. `s3a://my-bucket/my-warehouse` 
+2. For `HiveCatalog`, to store metadata also using S3A, specify Hadoop config `hive.metastore.warehouse.dir` to be an S3A path.
+3. Add [hadoop-aws](https://mvnrepository.com/artifact/org.apache.hadoop/hadoop-aws) as a runtime dependency, configure AWS settings based on [hadoop-aws documentation](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html) (make sure you check the version, s3a configuration varies a lot based on the version you use)   
+
+
+## AWS client configurations
+
+Many organizations have customized their way of obtaining AWS credential and region information and configuring details about AWS clients for features like proxy access, retry, etc.
+Therefore, we open a configurer interface for Iceberg users to plugin any client configuration in a centralized place. 
+Users can set the `client.configurer` property as the class of the custom configurer.
+
+For example, a configurer can do something like the following:
+
+```java
+package com.my.team;
+
+import software.amazon.awssdk.auth.credentials.ContainerCredentialsProvider;
+import software.amazon.awssdk.awscore.client.builder.AwsClientBuilder;
+import software.amazon.awssdk.services.s3.S3ClientBuilder;
+
+public class MyCustomClientConfigurer implements AwsClientConfigurer {
+
+  @Override
+  public <T extends AwsClientBuilder & AwsSyncClientBuilder> T configure(T clientBuilder) {
+    
+    // set some custom s3 configurations
+    if (clientBuilder instanceof S3ClientBuilder) {
+      S3ClientBuilder s3ClientBuilder = (S3ClientBuilder) clientBuilder;
+      // configure something
+    }
+
+    // set the same credential provider for all clients
+    clientBuilder.credentialsProvider(ContainerCredentialsProvider.builder().build());
+  }
+}
+```
+
+### AssumeRoleConfigurer
+
+As a common use case scenario, we provide `AssumeRoleConfigurer` as an example configurer. It has the following catalog properties:
+
+* `client.assume-role.arn`: role ARN to assume
+* `client.assume-role.timeout-sec`: seconds for an assume-role session, after the timeout a new session is automatically fetched by a STS client.
+* `client.assume-role.external-id`: optional external ID for the role to assume
+* `client.assume-role.region`: a region for all clients (except the STS client) to use
+
+When this configurer is used, a STS client is initialized with default [credentials chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html) and [region chain](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html),
+and all the other clients (Glue, DynamoDB, S3, etc.) will use the configured assume-role credential and region.
+
+## Run Iceberg on AWS
+
+[Amazon EMR](https://aws.amazon.com/emr/) is the most common platform to run Iceberg on AWS. EMR can provision clusters with [Spark](https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-spark.html) (EMR 6 for Spark 3, EMR 5 for Spark 2),

Review comment:
       Minor: did you have to pull down the aws sdk bundle to run sparksql in EMR? IIRC there's already some preinstalled aws libraries on EMR, if that's true we might be able to mention them here to save one step for the users. 

##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,212 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The first thing to note is that the `iceberg-aws` module is not bundled with any engine runtime.
+To use any features described in later sections, you need to include the following packages by yourself:
+
+* the [iceberg AWS package](https://mvnrepository.com/artifact/org.apache.iceberg/iceberg-aws)
+* the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), or individual AWS client packages if you would like to have a minimum dependency footprint. (please note that we use the new AWS v2 SDK instead of v1)
+
+For example, in Spark 3, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,org.apache.iceberg:iceberg-aws-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \
+    --conf spark.sql.catalog.my_catalog.gluecatalog.lock.table=myGlueLockTable
+```
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg `Namespace` is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg `Table` is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+an Iceberg `Snapshot` is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-TableVersion). 
+You can start using Glue catalog by specifying the `catalog-impl` as `org.apache.iceberg.aws.glue.GlueCatalog`. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+It is very common for an organization to store all the tables in a single Glue catalog in a single AWS account and run data computation in many different accounts. 
+In this case, you need to specify a Glue catalog ID when initializing `GlueCatalog`.
+The Glue catalog ID you should use is the AWS account ID.
+This is because in each AWS account, there is a single Glue catalog in each AWS region,
+but the region is pre-determined by the Glue web client that is making the call.
+If you would like to access a Glue catalog in a different region, you should configure you AWS client, see more details in [AWS client configuration](#aws-client-configurations).
+It is also common to [assume a role](https://docs.aws.amazon.com/STS/latest/APIReference/API_AssumeRole.html) when having cross-account access. See [AssumeRoleConfigurer](#assumeroleconfigurer) for how to set up assume role credentials in Iceberg.
+
+### Skip Archive
+
+By default, Glue will store 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 `` to false.

Review comment:
       Forgot to update the string to be set? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553687259



##########
File path: site/docs/configuration.md
##########
@@ -35,6 +35,20 @@ The properties can be manually constructed or passed in from a compute engine li
 Spark uses its session properties as catalog properties, see more details in the [Spark configuration](#spark-configuration) section.
 Flink passes in catalog properties through `CREATE CATALOG` statement, see more details in the [Flink](../flink/#creating-catalogs-and-using-catalogs) section.
 
+### Lock catalog properties
+
+Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.
+
+| Property                          | Default            | Description                                            |
+| --------------------------------- | ------------------ | ------------------------------------------------------ |
+| lock.impl                         | null               | a custom implementation of the lock manager, the actual interface depends on the catalog used  |
+| lock.table                        | null               | an optional auxiliary table for locking                |

Review comment:
       Thanks for pointing that out. You're right that it is optional here. Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jackye1995 commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553496098



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,170 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+However, the AWS clients are not bundled so that you can use the same version as your application.
+Please note that we use the new AWS v2 SDK instead of v1.
+You can choose to the [AWS SDK bundle](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS) if you would like to have a minimum dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
+    --conf spark.sql.catalog.my_catalog.catalog-impl=org.apache.iceberg.aws.glue.GlueCatalog \
+    --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my-key-prefix \

Review comment:
       good point, let me add that section




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1891: AWS: documentation page for AWS module

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1891:
URL: https://github.com/apache/iceberg/pull/1891#discussion_r553673431



##########
File path: site/docs/aws.md
##########
@@ -0,0 +1,187 @@
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+ 
+# Iceberg AWS Integrations
+
+Iceberg provides integration with different AWS services through the `iceberg-aws` module. 
+This section describes how to use Iceberg with AWS.
+
+## Runtime Packages
+
+The `iceberg-aws` module is bundled with Spark and Flink engine runtimes.
+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](https://mvnrepository.com/artifact/software.amazon.awssdk/bundle), 
+or individual AWS client packages (Glue, S3, DynamoDB, KMS, STS) if you would like to have a minimal dependency footprint.
+
+For example, to use AWS features with Spark 3 and AWS clients version 2.15.40, you can start the SQL shell with:
+
+```sh
+spark-sql --packages org.apache.iceberg:iceberg-spark3-runtime:0.11.0,software.amazon.awssdk:bundle:2.15.40 \
+    --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.lock.impl=org.apache.iceberg.aws.glue.DynamoLockManager \
+    --conf spark.sql.catalog.my_catalog.lock.table=myGlueLockTable
+```
+
+As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle dependency with its version as `2.15.40`.
+
+## Glue Catalog
+
+Iceberg enables the use of [AWS Glue](https://aws.amazon.com/glue) as the `Catalog` implementation.
+When used, an Iceberg namespace is stored as a [Glue Database](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-databases.html), 
+an Iceberg table is stored as a [Glue Table](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html),
+and every Iceberg table version is stored as a [Glue TableVersion](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-api-catalog-tables.html#aws-glue-api-catalog-tables-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 [runtime packages](#runtime-packages) section above. 
+More details about loading the catalog can be found in individual engine pages, such as [Spark](../spark/#loading-a-custom-catalog) and [Flink](../flink/#creating-catalogs-and-using-catalogs).
+
+### Glue Catalog ID
+
+You can specify the Glue catalog ID through `gluecatalog.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](#aws-client-customization).
+
+### Skip Archive
+
+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 `gluecatalog.skip-archive` to true.
+For more details, please read [Glue Quotas](https://docs.aws.amazon.com/general/latest/gr/glue.html) and the [UpdateTable API](https://docs.aws.amazon.com/glue/latest/webapi/API_UpdateTable.html).
+
+### DynamoDB for commit locking
+
+Glue does not have a strong guarantee over concurrent updates to a table. 
+Although it throws `ConcurrentModificationException` when detecting two processes updating a table at the same time,
+there is no guarantee that one update would not clobber the other update.
+Therefore, DynamoDB can be used for Glue, so that for every commit, 
+`GlueCatalog` first obtains a lock using a helper DynamoDB table and then try to safely modify the Glue table.
+
+To enable this feature, use the lock related catalog properties:
+
+1. Set `lock.impl` as `org.apache.iceberg.aws.glue.DynamoLockManager`.
+2. Set `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](https://aws.amazon.com/blogs/aws/amazon-dynamodb-on-demand-no-capacity-planning-and-pay-per-request-pricing).
+
+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](../configuration/#lock-catalog-properties). 
+
+### Warehouse Location
+
+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 through the mechanism described in the [custom FileIO](../custom-catalog/#custom-file-io-implementation) section.
+
+### Table Location
+
+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 root location can be changed at both namespace and table level.
+
+To use a different path prefix for all tables under a namespace, use 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 example, 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 be the desired root path value.
+
+## S3 FileIO
+
+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.
+
+### Progressive Multipart Upload
+
+`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                                            |
+| --------------------------------- | -------------------------------------------------- | ------------------------------------------------------ |
+| s3fileio.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)  |
+| s3fileio.multipart.part.size      | 32MB                                               | the size of a single part for multipart upload requests  |
+| s3fileio.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  |
+| s3fileio.staging.dir              | `java.io.tmpdir` property value                    | the directory to hold temporary files  |

Review comment:
       Is there anything else under staging, or should this be `staging-dir`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org