Published versioned 1.4.1 docs
diff --git a/docs/content/aws.md b/docs/content/aws.md
index 2f8e191..3b5a48e 100644
--- a/docs/content/aws.md
+++ b/docs/content/aws.md
@@ -4,6 +4,7 @@
 menu:
     main:
         parent: Integrations
+        identifier: aws_integration
         weight: 0
 ---
 <!--
@@ -36,10 +37,10 @@
 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.
 
-All the default AWS clients use the [URL Connection HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/url-connection-client)
+All the default AWS clients use the [Apache HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/apache-client)
 for HTTP connection management.
 This dependency is not part of the AWS SDK bundle and needs to be added separately.
-To choose a different HTTP client library such as [Apache HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/apache-client),
+To choose a different HTTP client library such as [URL Connection HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/url-connection-client),
 see the section [client customization](#aws-client-customization) for more details.
 
 All the AWS module features can be loaded through custom catalog properties,
@@ -48,25 +49,11 @@
 
 ### Spark
 
-For example, to use AWS features with Spark 3.3 (with scala 2.12) and AWS clients version 2.20.18, you can start the Spark SQL shell with:
+For example, to use AWS features with Spark 3.4 (with scala 2.12) and AWS clients version 2.20.131 (which is packaged in the `iceberg-aws-bundle`), you can start the Spark SQL shell with:
 
 ```sh
-# add Iceberg dependency
-ICEBERG_VERSION={{% icebergVersion %}}
-DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:$ICEBERG_VERSION"
-
-# add AWS dependnecy
-AWS_SDK_VERSION=2.20.18
-AWS_MAVEN_GROUP=software.amazon.awssdk
-AWS_PACKAGES=(
-    "bundle"
-)
-for pkg in "${AWS_PACKAGES[@]}"; do
-    DEPENDENCIES+=",$AWS_MAVEN_GROUP:$pkg:$AWS_SDK_VERSION"
-done
-
 # start Spark SQL client shell
-spark-sql --packages $DEPENDENCIES \
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.4_2.12:{{% icebergVersion %}},org.apache.iceberg:iceberg-aws-bundle:{{% icebergVersion %}} \
     --conf spark.sql.defaultCatalog=my_catalog \
     --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \
     --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket/my/key/prefix \
@@ -74,7 +61,7 @@
     --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO
 ```
 
-As you can see, In the shell command, we use `--packages` to specify the additional AWS bundle and HTTP client dependencies with their version as `2.20.18`.
+As you can see, In the shell command, we use `--packages` to specify the additional `iceberg-aws-bundle` that contains all relevant AWS dependencies.
 
 ### Flink
 
@@ -86,21 +73,12 @@
 MAVEN_URL=https://repo1.maven.org/maven2
 ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg
 wget $ICEBERG_MAVEN_URL/iceberg-flink-runtime/$ICEBERG_VERSION/iceberg-flink-runtime-$ICEBERG_VERSION.jar
-
-# download AWS dependnecy
-AWS_SDK_VERSION=2.20.18
-AWS_MAVEN_URL=$MAVEN_URL/software/amazon/awssdk
-AWS_PACKAGES=(
-    "bundle"
-)
-for pkg in "${AWS_PACKAGES[@]}"; do
-    wget $AWS_MAVEN_URL/$pkg/$AWS_SDK_VERSION/$pkg-$AWS_SDK_VERSION.jar
-done
+wget $ICEBERG_MAVEN_URL/iceberg-aws-bundle/$ICEBERG_VERSION/iceberg-aws-bundle-$ICEBERG_VERSION.jar
 
 # start Flink SQL client shell
 /path/to/bin/sql-client.sh embedded \
     -j iceberg-flink-runtime-$ICEBERG_VERSION.jar \
-    -j bundle-$AWS_SDK_VERSION.jar \
+    -j iceberg-aws-bundle-$ICEBERG_VERSION.jar \
     shell
 ```
 
@@ -172,12 +150,12 @@
 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, 
+If the Glue catalog is in a different region, you should configure your AWS client to point to the correct region, 
 see more details in [AWS client customization](#aws-client-customization).
 
 #### Skip Archive
 
-AWS Glue has the ability to archive older table versions and a user can rollback the table to any historical version if needed.
+AWS Glue has the ability to archive older table versions and a user can roll back the table to any historical version if needed.
 By default, the Iceberg Glue Catalog will skip the archival of older table versions.
 If a user wishes to archive older table versions, they can set `glue.skip-archive` to false.
 Do note for streaming ingestion into Iceberg tables, setting `glue.skip-archive` to false will quickly create a lot of Glue table versions.
@@ -186,8 +164,8 @@
 #### Skip Name Validation
 
 Allow user to skip name validation for table name and namespaces.
-It is recommended to stick to Glue best practice in
-https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html to make sure operations are Hive compatible.
+It is recommended to stick to [Glue best practices](https://docs.aws.amazon.com/athena/latest/ug/glue-best-practices.html)
+to make sure operations are Hive compatible.
 This is only added for users that have existing conventions using non-standard characters. When database name
 and table name validation are skipped, there is no guarantee that downstream systems would all support the names.
 
@@ -197,9 +175,8 @@
 With optimistic locking, each table has a version id. 
 If users retrieve the table metadata, Iceberg records the version id of that table. 
 Users can update the table as long as the version ID on the server side remains unchanged. 
-If there is a version mismatch, it means that someone else has modified the table before you did. 
-The update attempt fails, because you have a stale version of the table. 
-If this happens, Iceberg refreshes the metadata and checks if there might be potential conflict. 
+Version mismatch occurs if someone else modified the table before you did, causing an update failure. 
+Iceberg then refreshes metadata and checks if there is a conflict.
 If there is no commit conflict, the operation will be retried.
 Optimistic locking guarantees atomic transaction of Iceberg tables in Glue.
 It also prevents others from accidentally overwriting your changes.
@@ -239,7 +216,7 @@
 PARTITIONED BY (category);
 ```
 
-For engines like Spark that supports the `LOCATION` keyword, the above SQL statement is equivalent to:
+For engines like Spark that support the `LOCATION` keyword, the above SQL statement is equivalent to:
 
 ```sql
 CREATE TABLE my_catalog.my_ns.my_table (
@@ -279,7 +256,7 @@
 
 This design has the following benefits:
 
-1. it avoids potential [hot partition issue](https://aws.amazon.com/premiumsupport/knowledge-center/dynamodb-table-throttled/) if there are heavy write traffic to the tables within the same namespace, because the partition key is at the table level
+1. it avoids potential [hot partition issue](https://aws.amazon.com/premiumsupport/knowledge-center/dynamodb-table-throttled/) if there are heavy write traffic to the tables within the same namespace because the partition key is at the table level
 2. namespace operations are clustered in a single partition to avoid affecting table commit operations
 3. a sort key to partition key reverse GSI is used for list table operation, and all other operations are single row ops or single partition query. No full table scan is needed for any operation in the catalog.
 4. a string UUID version field `v` is used instead of `updated_at` to avoid 2 processes committing at the same millisecond
@@ -288,25 +265,25 @@
 
 ### RDS JDBC Catalog
 
-Iceberg also supports JDBC catalog which uses a table in a relational database to manage Iceberg tables.
-You can configure to use JDBC catalog with relational database services like [AWS RDS](https://aws.amazon.com/rds).
+Iceberg also supports the JDBC catalog which uses a table in a relational database to manage Iceberg tables.
+You can configure to use the JDBC catalog with relational database services like [AWS RDS](https://aws.amazon.com/rds).
 Read [the JDBC integration page](../jdbc/#jdbc-catalog) for guides and examples about using the JDBC catalog.
-Read [this AWS documentation](https://docs.aws.amazon.com/AmazonRDS/latest/UserGuide/UsingWithRDS.IAMDBAuth.Connecting.Java.html) for more details about configuring JDBC catalog with IAM authentication. 
+Read [this AWS documentation](https://docs.aws.amazon.com/AmazonRDS/latest/UserGuide/UsingWithRDS.IAMDBAuth.Connecting.Java.html) for more details about configuring the JDBC catalog with IAM authentication. 
 
 ### Which catalog to choose?
 
-With all the available options, we offer the following guidance when choosing the right catalog to use for your application:
+With all the available options, we offer the following guidelines when choosing the right catalog to use for your application:
 
 1. if your organization has an existing Glue metastore or plans to use the AWS analytics ecosystem including Glue, [Athena](https://aws.amazon.com/athena), [EMR](https://aws.amazon.com/emr), [Redshift](https://aws.amazon.com/redshift) and [LakeFormation](https://aws.amazon.com/lake-formation), Glue catalog provides the easiest integration.
 2. if your application requires frequent updates to table or high read and write throughput (e.g. streaming write), Glue and DynamoDB catalog provides the best performance through optimistic locking.
 3. if you would like to enforce access control for tables in a catalog, Glue tables can be managed as an [IAM resource](https://docs.aws.amazon.com/service-authorization/latest/reference/list_awsglue.html), whereas DynamoDB catalog tables can only be managed through [item-level permission](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/specifying-conditions.html) which is much more complicated.
 4. if you would like to query tables based on table property information without the need to scan the entire catalog, DynamoDB catalog allows you to build secondary indexes for any arbitrary property field and provide efficient query performance.
 5. if you would like to have the benefit of DynamoDB catalog while also connect to Glue, you can enable [DynamoDB stream with Lambda trigger](https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/Streams.Lambda.Tutorial.html) to asynchronously update your Glue metastore with table information in the DynamoDB catalog. 
-6. if your organization already maintains an existing relational database in RDS or uses [serverless Aurora](https://aws.amazon.com/rds/aurora/serverless/) to manage tables, JDBC catalog provides the easiest integration.
+6. if your organization already maintains an existing relational database in RDS or uses [serverless Aurora](https://aws.amazon.com/rds/aurora/serverless/) to manage tables, the JDBC catalog provides the easiest integration.
 
 ## DynamoDb Lock Manager
 
-[Amazon DynamoDB](https://aws.amazon.com/dynamodb) can be used by `HadoopCatalog` or `HadoopTables`, so that for every commit,
+[Amazon DynamoDB](https://aws.amazon.com/dynamodb) can be used by `HadoopCatalog` or `HadoopTables` so that for every commit,
 the catalog first obtains a lock using a helper DynamoDB table and then try to safely modify the Iceberg table.
 This is necessary for a file system-based catalog to ensure atomic transaction in storages like S3 that do not provide file write mutual exclusion.
 
@@ -345,7 +322,7 @@
 
 * [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.
+* [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:
 
@@ -364,13 +341,13 @@
 ### Object Store File Layout
 
 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/).
-Data stored in S3 with a traditional Hive storage layout can face S3 request throttling as objects are stored under the same filepath prefix.
+Data stored in S3 with a traditional Hive storage layout can face S3 request throttling as objects are stored under the same file path prefix.
 
-Iceberg by default uses the Hive storage layout, but can be switched to use the `ObjectStoreLocationProvider`. 
-With `ObjectStoreLocationProvider`, a determenistic hash is generated for each stored file, with the hash appended 
+Iceberg by default uses the Hive storage layout but can be switched to use the `ObjectStoreLocationProvider`. 
+With `ObjectStoreLocationProvider`, a deterministic hash is generated for each stored file, with the hash appended 
 directly after the `write.data.path`. This ensures files written to s3 are equally distributed across multiple [prefixes](https://aws.amazon.com/premiumsupport/knowledge-center/s3-object-key-naming-pattern/) in the S3 bucket. Resulting in minimized throttling and maximized throughput for S3-related IO operations. When using `ObjectStoreLocationProvider` having a shared and short `write.data.path` across your Iceberg tables will improve performance.
 
-For more information on how S3 scales API QPS, checkout the 2018 re:Invent session on [Best Practices for Amazon S3 and Amazon S3 Glacier]( https://youtu.be/rHeTn9pHNKo?t=3219). At [53:39](https://youtu.be/rHeTn9pHNKo?t=3219) it covers how S3 scales/partitions & at [54:50](https://youtu.be/rHeTn9pHNKo?t=3290) it discusses the 30-60 minute wait time before new partitions are created.
+For more information on how S3 scales API QPS, check out the 2018 re:Invent session on [Best Practices for Amazon S3 and Amazon S3 Glacier]( https://youtu.be/rHeTn9pHNKo?t=3219). At [53:39](https://youtu.be/rHeTn9pHNKo?t=3219) it covers how S3 scales/partitions & at [54:50](https://youtu.be/rHeTn9pHNKo?t=3290) it discusses the 30-60 minute wait time before new partitions are created.
 
 To use the `ObjectStorageLocationProvider` add `'write.object-storage.enabled'=true` in the table's properties. 
 Below is an example Spark SQL command to create a table using the `ObjectStorageLocationProvider`:
@@ -411,8 +388,8 @@
 ### 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.
+As introduced in the previous sections, `S3FileIO` adopts the latest AWS clients and S3 features for optimized security and performance
+ and is thus recommended 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.
@@ -521,7 +498,7 @@
 ### S3 Dual-stack
 
 [S3 Dual-stack](https://docs.aws.amazon.com/AmazonS3/latest/userguide/dual-stack-endpoints.html) allows a client to access an S3 bucket through a dual-stack endpoint. 
-When clients make a request to a dual-stack endpoint, the bucket URL resolves to an IPv6 address if possible, otherwise fallback to IPv4.
+When clients request a dual-stack endpoint, the bucket URL resolves to an IPv6 address if possible, otherwise fallback to IPv4.
 
 To use S3 Dual-stack, we need to set `s3.dualstack-enabled` catalog property to `true` to enable `S3FileIO` to make dual-stack S3 calls.
 
@@ -555,14 +532,14 @@
 | client.assume-role.arn            | null, requires user input                | ARN of the role to assume, e.g. arn:aws:iam::123456789:role/myRoleToAssume  |
 | client.assume-role.region         | null, requires user input                | All AWS clients except the STS client will use the given region instead of the default region chain  |
 | client.assume-role.external-id    | null                                     | An optional [external ID](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_create_for-user_externalid.html)  |
-| client.assume-role.timeout-sec    | 1 hour                                   | Timeout of each assume role session. At the end of the timeout, a new set of role session credentials will be fetched through a STS client.  |
+| client.assume-role.timeout-sec    | 1 hour                                   | Timeout of each assume role session. At the end of the timeout, a new set of role session credentials will be fetched through an STS client.  |
 
 By using this client factory, an STS client is initialized with the default credential and region to assume the specified role.
 The Glue, S3 and DynamoDB clients are then initialized with the assume-role credential and region to access resources.
 Here is an example to start Spark shell with this client factory:
 
 ```shell
-spark-sql --packages org.apache.iceberg:iceberg-spark-runtime:{{% icebergVersion %}},software.amazon.awssdk:bundle:2.20.18 \
+spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.4_2.12:{{% icebergVersion %}},org.apache.iceberg:iceberg-aws-bundle:{{% icebergVersion %}} \
     --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 \
@@ -574,9 +551,9 @@
 ### HTTP Client Configurations
 AWS clients support two types of HTTP Client, [URL Connection HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/url-connection-client) 
 and [Apache HTTP Client](https://mvnrepository.com/artifact/software.amazon.awssdk/apache-client).
-By default, AWS clients use **URL Connection** HTTP Client to communicate with the service. 
-This HTTP client optimizes for minimum dependencies and startup latency but support less functionality than other implementations. 
-In contrast, Apache HTTP Client supports more functionalities and more customized settings, such as expect-continue handshake and TCP KeepAlive, at cost of extra dependency and additional startup latency. 
+By default, AWS clients use **Apache** HTTP Client to communicate with the service. 
+This HTTP client supports various functionalities and customized settings, such as expect-continue handshake and TCP KeepAlive, at the cost of extra dependency and additional startup latency.
+In contrast, URL Connection HTTP Client optimizes for minimum dependencies and startup latency but supports less functionality than other implementations.
 
 For more details of configuration, see sections [URL Connection HTTP Client Configurations](#url-connection-http-client-configurations) and [Apache HTTP Client Configurations](#apache-http-client-configurations).
 
@@ -606,15 +583,15 @@
 
 | Property                                              | Default                   | Description                                                                                                                                                                                                                                 |
 |-------------------------------------------------------|---------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| http-client.apache.socket-timeout-ms                  | null                      | An optional [socket timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#socketTimeout(java.time.Duration)) in milliseconds                                                  |
-| http-client.apache.connection-timeout-ms              | null                      | An optional [connection timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionTimeout(java.time.Duration)) in milliseconds                                          |
-| http-client.apache.connection-acquisition-timeout-ms  | null                      | An optional [connection acquisition timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionAcquisitionTimeout(java.time.Duration)) in milliseconds                   |
-| http-client.apache.connection-max-idle-time-ms        | null                      | An optional [connection max idle timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionMaxIdleTime(java.time.Duration)) in milliseconds                             |
-| http-client.apache.connection-time-to-live-ms         | null                      | An optional [connection time to live](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionTimeToLive(java.time.Duration)) in milliseconds                                  |
-| http-client.apache.expect-continue-enabled            | null, disabled by default | An optional `true/false` setting that decide whether to enable [expect continue](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#expectContinueEnabled(java.lang.Boolean))       |
-| http-client.apache.max-connections                    | null                      | An optional [max connections](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#maxConnections(java.lang.Integer))  in integer                                                     |
-| http-client.apache.tcp-keep-alive-enabled             | null, disabled by default | An optional `true/false` setting that decide whether to enable [tcp keep alive](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#tcpKeepAlive(java.lang.Boolean))                 |
-| http-client.apache.use-idle-connection-reaper-enabled | null, enabled by default  | An optional `true/false` setting that decide whether to [use idle connection reaper](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#useIdleConnectionReaper(java.lang.Boolean)) |
+| http-client.apache.socket-timeout-ms                  | null                      | An optional [socket timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#socketTimeout(java.time.Duration)) in milliseconds |
+| http-client.apache.connection-timeout-ms              | null                      | An optional [connection timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionTimeout(java.time.Duration)) in milliseconds |
+| http-client.apache.connection-acquisition-timeout-ms  | null                      | An optional [connection acquisition timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionAcquisitionTimeout(java.time.Duration)) in milliseconds |
+| http-client.apache.connection-max-idle-time-ms        | null                      | An optional [connection max idle timeout](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionMaxIdleTime(java.time.Duration)) in milliseconds |
+| http-client.apache.connection-time-to-live-ms         | null                      | An optional [connection time to live](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#connectionTimeToLive(java.time.Duration)) in milliseconds |
+| http-client.apache.expect-continue-enabled            | null, disabled by default | An optional `true/false` setting that controls whether [expect continue](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#expectContinueEnabled(java.lang.Boolean)) is enabled |
+| http-client.apache.max-connections                    | null                      | An optional [max connections](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#maxConnections(java.lang.Integer))  in integer       |
+| http-client.apache.tcp-keep-alive-enabled             | null, disabled by default | An optional `true/false` setting that controls whether [tcp keep alive](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#tcpKeepAlive(java.lang.Boolean)) is enabled |
+| http-client.apache.use-idle-connection-reaper-enabled | null, enabled by default  | An optional `true/false` setting that controls whether [use idle connection reaper](https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html#useIdleConnectionReaper(java.lang.Boolean)) is used |
 
 Users can use catalog properties to override the defaults. For example, to configure the max connections for Apache HTTP Client when starting a spark shell, one can add:
 ```shell
@@ -641,22 +618,17 @@
 ```sh
 #!/bin/bash
 
-AWS_SDK_VERSION=2.20.18
 ICEBERG_VERSION={{% icebergVersion %}}
 MAVEN_URL=https://repo1.maven.org/maven2
 ICEBERG_MAVEN_URL=$MAVEN_URL/org/apache/iceberg
-AWS_MAVEN_URL=$MAVEN_URL/software/amazon/awssdk
 # NOTE: this is just an example shared class path between Spark and Flink,
 #  please choose a proper class path for production.
 LIB_PATH=/usr/share/aws/aws-java-sdk/
 
-AWS_PACKAGES=(
-  "bundle"
-)
-
 ICEBERG_PACKAGES=(
   "iceberg-spark-runtime-3.3_2.12"
   "iceberg-flink-runtime"
+  "iceberg-aws-bundle"
 )
 
 install_dependencies () {
@@ -671,13 +643,12 @@
 }
 
 install_dependencies $LIB_PATH $ICEBERG_MAVEN_URL $ICEBERG_VERSION "${ICEBERG_PACKAGES[@]}"
-install_dependencies $LIB_PATH $AWS_MAVEN_URL $AWS_SDK_VERSION "${AWS_PACKAGES[@]}"
 ```
 
 ### AWS Glue
 
 [AWS Glue](https://aws.amazon.com/glue/) provides a serverless data integration service
-that could be used to perform read, write, update tasks against Iceberg tables.
+that could be used to perform read, write and update tasks against Iceberg tables.
 More details could be found [here](https://docs.aws.amazon.com/glue/latest/dg/aws-glue-programming-etl-format-iceberg.html).
 
 
diff --git a/docs/content/branching-and-tagging.md b/docs/content/branching-and-tagging.md
index f2d8606..2bff0a8 100644
--- a/docs/content/branching-and-tagging.md
+++ b/docs/content/branching-and-tagging.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_branching
         weight: 0
 ---
 
@@ -30,9 +31,9 @@
 
 ## Overview
 
-Iceberg table metadata maintains a log of snapshots which represent the changes applied to a table.
+Iceberg table metadata maintains a snapshot log, which represents the changes applied to a table.
 Snapshots are fundamental in Iceberg as they are the basis for reader isolation and time travel queries.
-For controlling metadata size and storage costs, Iceberg provides snapshot lifecycle management procedures such as [`expire_snapshots`](../../spark/spark-procedures/#expire-snapshots) for removing unused snapshots and no longer neccessary data files based on table snapshot retention properties.
+For controlling metadata size and storage costs, Iceberg provides snapshot lifecycle management procedures such as [`expire_snapshots`](../spark-procedures/#expire-snapshots) for removing unused snapshots and no longer necessary data files based on table snapshot retention properties.
 
 **For more sophisticated snapshot lifecycle management, Iceberg supports branches and tags which are named references to snapshots with their own independent lifecycles. This lifecycle is controlled by branch and tag level retention policies.** 
 Branches are independent lineages of snapshots and point to the head of the lineage. 
@@ -53,32 +54,32 @@
 
 ![Historical Tags](../img/historical-snapshot-tag.png)
 
-The above diagram demonstrates retaininig important historical snapshot with the following retention policy, defined 
+The above diagram demonstrates retaining important historical snapshot with the following retention policy, defined 
 via Spark SQL.
 
 1. Retain 1 snapshot per week for 1 month. This can be achieved by tagging the weekly snapshot and setting the tag retention to be a month.
 snapshots will be kept, and the branch reference itself will be retained for 1 week. 
 ```sql
 -- Create a tag for the first end of week snapshot. Retain the snapshot for a week
-ALTER TABLE prod.db.table CREATE TAG 'EOW-01' AS OF VERSION 7 RETAIN 7 DAYS
+ALTER TABLE prod.db.table CREATE TAG `EOW-01` AS OF VERSION 7 RETAIN 7 DAYS
 ```
 
 2. Retain 1 snapshot per month for 6 months. This can be achieved by tagging the monthly snapshot and setting the tag retention to be 6 months.
 ```sql
 -- Create a tag for the first end of month snapshot. Retain the snapshot for 6 months
-ALTER TABLE prod.db.table CREATE TAG 'EOM-01' AS OF VERSION 30 RETAIN 180 DAYS
+ALTER TABLE prod.db.table CREATE TAG `EOM-01` AS OF VERSION 30 RETAIN 180 DAYS
 ```
 
 3. Retain 1 snapshot per year forever. This can be achieved by tagging the annual snapshot. The default retention for branches and tags is forever.
 ```sql
 -- Create a tag for the end of the year and retain it forever.
-ALTER TABLE prod.db.table CREATE TAG 'EOY-2023' AS OF VERSION 365
+ALTER TABLE prod.db.table CREATE TAG `EOY-2023` AS OF VERSION 365
 ```
 
 4. Create a temporary "test-branch" which is retained for 7 days and the latest 2 snapshots on the branch are retained.
 ```sql
 -- Create a branch "test-branch" which will be retained for 7 days along with the  latest 2 snapshots
-ALTER TABLE prod.db.table CREATE BRANCH test-branch RETAIN 7 DAYS WITH RETENTION 2 SNAPSHOTS
+ALTER TABLE prod.db.table CREATE BRANCH `test-branch` RETAIN 7 DAYS WITH SNAPSHOT RETENTION 2 SNAPSHOTS
 ```
 
 ### Audit Branch
@@ -90,7 +91,7 @@
 1. First ensure `write.wap.enabled` is set.
 ```sql
 ALTER TABLE db.table SET TBLPROPERTIES (
-    'write.wap.enabled''true'
+    'write.wap.enabled'='true'
 )
 ```
 2. Create `audit-branch` starting from snapshot 3, which will be written to and retained for 1 week.
@@ -100,13 +101,13 @@
 3. Writes are performed on a separate `audit-branch` independent from the main table history.
 ```sql
 -- WAP Branch write
-SET spark.wap.branch = 'audit-branch'
+SET spark.wap.branch = audit-branch
 INSERT INTO prod.db.table VALUES (3, 'c')
 ```
 4. A validation workflow can validate (e.g. data quality) the state of `audit-branch`.
 5. After validation, the main branch can be `fastForward` to the head of `audit-branch` to update the main table state.
-```java
-table.manageSnapshots().fastForward("main", "audit-branch").commit()
+```sql
+CALL catalog_name.system.fast_forward('prod.db.table', 'main', 'audit-branch')
 ```
 6. The branch reference will be removed when `expireSnapshots` is run 1 week later.
 
@@ -114,7 +115,7 @@
 
 Creating, querying and writing to branches and tags are supported in the Iceberg Java library, and in Spark and Flink engine integrations.
 
-- [Iceberg Java Library](../../java-api-quickstart/#branching-and-tagging)
+- [Iceberg Java Library](../java-api-quickstart/#branching-and-tagging)
 - [Spark DDLs](../spark-ddl/#branching-and-tagging-ddl)
 - [Spark Reads](../spark-queries/#time-travel)
 - [Spark Branch Writes](../spark-writes/#writing-to-branches)
diff --git a/docs/content/configuration.md b/docs/content/configuration.md
index 8efd8fc..7c568e7 100644
--- a/docs/content/configuration.md
+++ b/docs/content/configuration.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_configuration
         weight: 0
 ---
 <!--
@@ -46,51 +47,51 @@
 
 ### Write properties
 
-| Property                                            | Default                    | Description                                                                                                                                                                                       |
-|-----------------------------------------------------|----------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| write.format.default                                | parquet                    | Default file format for the table; parquet, avro, or orc                                                                                                                                          |
-| write.delete.format.default                         | data file format           | Default delete file format for the table; parquet, avro, or orc                                                                                                                                   |
-| write.parquet.row-group-size-bytes                  | 134217728 (128 MB)         | Parquet row group size                                                                                                                                                                            |
-| write.parquet.page-size-bytes                       | 1048576 (1 MB)             | Parquet page size                                                                                                                                                                                 |
-| write.parquet.page-row-limit                        | 20000                      | Parquet page row limit                                                                                                                                                                            |
-| write.parquet.dict-size-bytes                       | 2097152 (2 MB)             | Parquet dictionary page size                                                                                                                                                                      |
-| write.parquet.compression-codec                     | gzip                       | Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed                                                                                                                          |
-| write.parquet.compression-level                     | null                       | Parquet compression level                                                                                                                                                                         |
-| write.parquet.bloom-filter-enabled.column.col1      | (not set)                  | Enables writing a bloom filter for the column: col1                                                                                                                                               |
-| write.parquet.bloom-filter-max-bytes                | 1048576 (1 MB)             | The maximum number of bytes for a bloom filter bitset                                                                                                                                             |
-| write.avro.compression-codec                        | gzip                       | Avro compression codec: gzip(deflate with 9 level), zstd, snappy, uncompressed                                                                                                                    |
-| write.avro.compression-level                        | null                       | Avro compression level                                                                                                                                                                            |
-| write.orc.stripe-size-bytes                         | 67108864 (64 MB)           | Define the default ORC stripe size, in bytes                                                                                                                                                      |
-| write.orc.block-size-bytes                          | 268435456 (256 MB)         | Define the default file system block size for ORC files                                                                                                                                           |
-| write.orc.compression-codec                         | zlib                       | ORC compression codec: zstd, lz4, lzo, zlib, snappy, none                                                                                                                                         |
-| write.orc.compression-strategy                      | speed                      | ORC compression strategy: speed, compression                                                                                                                                                      |
-| write.orc.bloom.filter.columns                      | (not set)                  | Comma separated list of column names for which a Bloom filter must be created                                                                                                                     |
-| write.orc.bloom.filter.fpp                          | 0.05                       | False positive probability for Bloom filter (must > 0.0 and < 1.0)                                                                                                                                |
-| write.location-provider.impl                        | null                       | Optional custom implementation for LocationProvider                                                                                                                                               |
-| write.metadata.compression-codec                    | none                       | Metadata compression codec; none or gzip                                                                                                                                                          |
-| write.metadata.metrics.max-inferred-column-defaults | 100                        | Defines the maximum number of columns for which metrics are collected                                                                                                                             |
-| write.metadata.metrics.default                      | truncate(16)               | Default metrics mode for all columns in the table; none, counts, truncate(length), or full                                                                                                        |
-| write.metadata.metrics.column.col1                  | (not set)                  | Metrics mode for column 'col1' to allow per-column tuning; none, counts, truncate(length), or full                                                                                                |
-| write.target-file-size-bytes                        | 536870912 (512 MB)         | Controls the size of files generated to target about this many bytes                                                                                                                              |
-| write.delete.target-file-size-bytes                 | 67108864 (64 MB)           | Controls the size of delete files generated to target about this many bytes                                                                                                                       |
-| write.distribution-mode                             | none                       | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder |
-| write.delete.distribution-mode                      | hash                       | Defines distribution of write delete data                                                                                                                                                         |
-| write.update.distribution-mode                      | hash                       | Defines distribution of write update data                                                                                                                                                         |
-| write.merge.distribution-mode                       | none                       | Defines distribution of write merge data                                                                                                                                                          |
-| write.wap.enabled                                   | false                      | Enables write-audit-publish writes                                                                                                                                                                |
-| write.summary.partition-limit                       | 0                          | Includes partition-level summary stats in snapshot summaries if the changed partition count is less than this limit                                                                               |
-| write.metadata.delete-after-commit.enabled          | false                      | Controls whether to delete the oldest **tracked** version metadata files after commit                                                                                                             |
-| write.metadata.previous-versions-max                | 100                        | The max number of previous version metadata files to keep before deleting after commit                                                                                                            |
-| write.spark.fanout.enabled                          | false                      | Enables the fanout writer in Spark that does not require data to be clustered; uses more memory                                                                                                   |
-| write.object-storage.enabled                        | false                      | Enables the object storage location provider that adds a hash component to file paths                                                                                                             |
-| write.data.path                                     | table location + /data     | Base location for data files                                                                                                                                                                      |
-| write.metadata.path                                 | table location + /metadata | Base location for metadata files                                                                                                                                                                  |
-| write.delete.mode                                   | copy-on-write              | Mode used for delete commands: copy-on-write or merge-on-read (v2 only)                                                                                                                           |
-| write.delete.isolation-level                        | serializable               | Isolation level for delete commands: serializable or snapshot                                                                                                                                     |
-| write.update.mode                                   | copy-on-write              | Mode used for update commands: copy-on-write or merge-on-read (v2 only)                                                                                                                           |
-| write.update.isolation-level                        | serializable               | Isolation level for update commands: serializable or snapshot                                                                                                                                     |
-| write.merge.mode                                    | copy-on-write              | Mode used for merge commands: copy-on-write or merge-on-read (v2 only)                                                                                                                            |
-| write.merge.isolation-level                         | serializable               | Isolation level for merge commands: serializable or snapshot                                                                                                                                      |
+| Property                                             | Default                     | Description                                                                                                                                                                                       |
+|------------------------------------------------------|-----------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| write.format.default                                 | parquet                     | Default file format for the table; parquet, avro, or orc                                                                                                                                          |
+| write.delete.format.default                          | data file format            | Default delete file format for the table; parquet, avro, or orc                                                                                                                                   |
+| write.parquet.row-group-size-bytes                   | 134217728 (128 MB)          | Parquet row group size                                                                                                                                                                            |
+| write.parquet.page-size-bytes                        | 1048576 (1 MB)              | Parquet page size                                                                                                                                                                                 |
+| write.parquet.page-row-limit                         | 20000                       | Parquet page row limit                                                                                                                                                                            |
+| write.parquet.dict-size-bytes                        | 2097152 (2 MB)              | Parquet dictionary page size                                                                                                                                                                      |
+| write.parquet.compression-codec                      | gzip                        | Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed                                                                                                                          |
+| write.parquet.compression-level                      | null                        | Parquet compression level                                                                                                                                                                         |
+| write.parquet.bloom-filter-enabled.column.col1       | (not set)                   | Hint to parquet to write a bloom filter for the column: col1                                                                                                                                      |
+| write.parquet.bloom-filter-max-bytes                 | 1048576 (1 MB)              | The maximum number of bytes for a bloom filter bitset                                                                                                                                             |
+| write.avro.compression-codec                         | gzip                        | Avro compression codec: gzip(deflate with 9 level), zstd, snappy, uncompressed                                                                                                                    |
+| write.avro.compression-level                         | null                        | Avro compression level                                                                                                                                                                            |
+| write.orc.stripe-size-bytes                          | 67108864 (64 MB)            | Define the default ORC stripe size, in bytes                                                                                                                                                      |
+| write.orc.block-size-bytes                           | 268435456 (256 MB)          | Define the default file system block size for ORC files                                                                                                                                           |
+| write.orc.compression-codec                          | zlib                        | ORC compression codec: zstd, lz4, lzo, zlib, snappy, none                                                                                                                                         |
+| write.orc.compression-strategy                       | speed                       | ORC compression strategy: speed, compression                                                                                                                                                      |
+| write.orc.bloom.filter.columns                       | (not set)                   | Comma separated list of column names for which a Bloom filter must be created                                                                                                                     |
+| write.orc.bloom.filter.fpp                           | 0.05                        | False positive probability for Bloom filter (must > 0.0 and < 1.0)                                                                                                                                |
+| write.location-provider.impl                         | null                        | Optional custom implementation for LocationProvider                                                                                                                                               |
+| write.metadata.compression-codec                     | none                        | Metadata compression codec; none or gzip                                                                                                                                                          |
+| write.metadata.metrics.max-inferred-column-defaults  | 100                         | Defines the maximum number of columns for which metrics are collected                                                                                                                             |
+| write.metadata.metrics.default                       | truncate(16)                | Default metrics mode for all columns in the table; none, counts, truncate(length), or full                                                                                                        |
+| write.metadata.metrics.column.col1                   | (not set)                   | Metrics mode for column 'col1' to allow per-column tuning; none, counts, truncate(length), or full                                                                                                |
+| write.target-file-size-bytes                         | 536870912 (512 MB)          | Controls the size of files generated to target about this many bytes                                                                                                                              |
+| write.delete.target-file-size-bytes                  | 67108864 (64 MB)            | Controls the size of delete files generated to target about this many bytes                                                                                                                       |
+| write.distribution-mode                              | none                        | Defines distribution of write data: __none__: don't shuffle rows; __hash__: hash distribute by partition key ; __range__: range distribute by partition key or sort key if table has an SortOrder |
+| write.delete.distribution-mode                       | hash                        | Defines distribution of write delete data                                                                                                                                                         |
+| write.update.distribution-mode                       | hash                        | Defines distribution of write update data                                                                                                                                                         |
+| write.merge.distribution-mode                        | none                        | Defines distribution of write merge data                                                                                                                                                          |
+| write.wap.enabled                                    | false                       | Enables write-audit-publish writes                                                                                                                                                                |
+| write.summary.partition-limit                        | 0                           | Includes partition-level summary stats in snapshot summaries if the changed partition count is less than this limit                                                                               |
+| write.metadata.delete-after-commit.enabled           | false                       | Controls whether to delete the oldest **tracked** version metadata files after commit                                                                                                             |
+| write.metadata.previous-versions-max                 | 100                         | The max number of previous version metadata files to keep before deleting after commit                                                                                                            |
+| write.spark.fanout.enabled                           | false                       | Enables the fanout writer in Spark that does not require data to be clustered; uses more memory                                                                                                   |
+| write.object-storage.enabled                         | false                       | Enables the object storage location provider that adds a hash component to file paths                                                                                                             |
+| write.data.path                                      | table location + /data      | Base location for data files                                                                                                                                                                      |
+| write.metadata.path                                  | table location + /metadata  | Base location for metadata files                                                                                                                                                                  |
+| write.delete.mode                                    | copy-on-write               | Mode used for delete commands: copy-on-write or merge-on-read (v2 only)                                                                                                                           |
+| write.delete.isolation-level                         | serializable                | Isolation level for delete commands: serializable or snapshot                                                                                                                                     |
+| write.update.mode                                    | copy-on-write               | Mode used for update commands: copy-on-write or merge-on-read (v2 only)                                                                                                                           |
+| write.update.isolation-level                         | serializable                | Isolation level for update commands: serializable or snapshot                                                                                                                                     |
+| write.merge.mode                                     | copy-on-write               | Mode used for merge commands: copy-on-write or merge-on-read (v2 only)                                                                                                                            |
+| write.merge.isolation-level                          | serializable                | Isolation level for merge commands: serializable or snapshot                                                                                                                                      |
 
 ### Table behavior properties
 
@@ -138,6 +139,7 @@
 | clients                           | 2                  | client pool size                                       |
 | cache-enabled                     | true               | Whether to cache catalog entries |
 | cache.expiration-interval-ms      | 30000              | How long catalog entries are locally cached, in milliseconds; 0 disables caching, negative values disable expiration |
+| metrics-reporter-impl | org.apache.iceberg.metrics.LoggingMetricsReporter | Custom `MetricsReporter` implementation to use in a catalog. See the [Metrics reporting](metrics-reporting) section for additional details |
 
 `HadoopCatalog` and `HiveCatalog` can access the properties in their constructors.
 Any other custom catalog can access the properties by implementing `Catalog.initialize(catalogName, catalogProperties)`.
@@ -178,8 +180,21 @@
 | iceberg.hive.lock-heartbeat-interval-ms   | 240000 (4 min)  | The heartbeat interval for the HMS locks.                                    |
 | iceberg.hive.metadata-refresh-max-retries | 2               | Maximum number of retries when the metadata file is missing                  |
 | iceberg.hive.table-level-lock-evict-ms    | 600000 (10 min) | The timeout for the JVM table lock is                                        |
+| iceberg.engine.hive.lock-enabled          | true            | Use HMS locks to ensure atomicity of commits                                 |
 
 Note: `iceberg.hive.lock-check-max-wait-ms` and `iceberg.hive.lock-heartbeat-interval-ms` should be less than the [transaction timeout](https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout) 
 of the Hive Metastore (`hive.txn.timeout` or `metastore.txn.timeout` in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the 
 Hive Metastore before the lock is retried from Iceberg.
 
+Warn: Setting `iceberg.engine.hive.lock-enabled`=`false` will cause HiveCatalog to commit to tables without using Hive locks.
+This should only be set to `false` if all following conditions are met:
+ - [HIVE-26882](https://issues.apache.org/jira/browse/HIVE-26882)
+is available on the Hive Metastore server
+ - All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later
+ - All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit.
+
+**Failing to ensure these conditions risks corrupting the table.**
+
+Even with `iceberg.engine.hive.lock-enabled` set to `false`, a HiveCatalog can still use locks for individual tables by setting the table property `engine.hive.lock-enabled`=`true`.
+This is useful in the case where other HiveCatalogs cannot be upgraded and set to commit without using Hive locks.
+
diff --git a/docs/content/dell.md b/docs/content/dell.md
index 173f2d1..af484f9 100644
--- a/docs/content/dell.md
+++ b/docs/content/dell.md
@@ -4,6 +4,7 @@
 menu:
     main:
         parent: Integrations
+        identifier: dell_integration
         weight: 0
 ---
 <!--
diff --git a/docs/content/delta-lake-migration.md b/docs/content/delta-lake-migration.md
index 2c1dddf..b0e7b22 100644
--- a/docs/content/delta-lake-migration.md
+++ b/docs/content/delta-lake-migration.md
@@ -4,6 +4,7 @@
 menu:
   main:
     parent: "Migration"
+    identifier: delta_lake_migration
     weight: 300
 ---
 <!--
diff --git a/docs/content/evolution.md b/docs/content/evolution.md
index 3f3f18e..7baf463 100644
--- a/docs/content/evolution.md
+++ b/docs/content/evolution.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_evolution
         weight: 0
 ---
 <!--
diff --git a/docs/content/flink-actions.md b/docs/content/flink-actions.md
index 1fc5bb8..32bc3b9 100644
--- a/docs/content/flink-actions.md
+++ b/docs/content/flink-actions.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_actions
         weight: 500
 ---
 <!--
diff --git a/docs/content/flink-configuration.md b/docs/content/flink-configuration.md
index 8974115..1877de2 100644
--- a/docs/content/flink-configuration.md
+++ b/docs/content/flink-configuration.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_configuration
         weight: 600
 ---
 <!--
@@ -86,7 +87,7 @@
     .tableLoader(TableLoader.fromCatalog(...))
     .assignerFactory(new SimpleSplitAssignerFactory())
     .streaming(true)
-    .streamingStartingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT)
+    .streamingStartingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID)
     .startSnapshotId(3821550127947089987L)
     .monitorInterval(Duration.ofMillis(10L)) // or .set("monitor-interval", "10s") \ set(FlinkReadOptions.MONITOR_INTERVAL, "10s")
     .build()
@@ -110,27 +111,28 @@
 
 `Read option` has the highest priority, followed by `Flink configuration` and then `Table property`.
 
-| Read option                 | Flink configuration                           | Table property               | Default                          | Description                                                  |
-| --------------------------- | --------------------------------------------- | ---------------------------- | -------------------------------- | ------------------------------------------------------------ |
-| snapshot-id                 | N/A                                           | N/A                          | null                             | For time travel in batch mode. Read data from the specified snapshot-id. |
-| case-sensitive              | connector.iceberg.case-sensitive              | N/A                          | false                            | If true, match column name in a case sensitive way.          |
-| as-of-timestamp             | N/A                                           | N/A                          | null                             | For time travel in batch mode. Read data from the most recent snapshot as of the given time in milliseconds. |
-| starting-strategy           | connector.iceberg.starting-strategy           | N/A                          | INCREMENTAL_FROM_LATEST_SNAPSHOT | Starting strategy for streaming execution. TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode. The incremental mode starts from the current snapshot exclusive. INCREMENTAL_FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive. INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. If the timestamp is between two snapshots, it should start from the snapshot after the timestamp. Just for FIP27 Source. |
-| start-snapshot-timestamp    | N/A                                           | N/A                          | null                             | Start to read data from the most recent snapshot as of the given time in milliseconds. |
-| start-snapshot-id           | N/A                                           | N/A                          | null                             | Start to read data from the specified snapshot-id.           |
-| end-snapshot-id             | N/A                                           | N/A                          | The latest snapshot id           | Specifies the end snapshot.  
-| branch                     | N/A                                            | N/A             | main       | Specifies the branch to read from in batch mode
-| tag                        | N/A                                            | N/A             | null       | Specifies the tag to read from in batch mode
-| start-tag                  | N/A                                            | N/A             | null       | Specifies the starting tag to read from for incremental reads
-| end-tag                    | N/A                                            | N/A             | null       | Specifies the ending tag to to read from for incremental reads                                |
-| split-size                  | connector.iceberg.split-size                  | read.split.target-size       | 128 MB                           | Target size when combining input splits.                     |
-| split-lookback              | connector.iceberg.split-file-open-cost        | read.split.planning-lookback | 10                               | Number of bins to consider when combining input splits.      |
-| split-file-open-cost        | connector.iceberg.split-file-open-cost        | read.split.open-file-cost    | 4MB                              | The estimated cost to open a file, used as a minimum weight when combining splits. |
-| streaming                   | connector.iceberg.streaming                   | N/A                          | false                            | Sets whether the current task runs in streaming or batch mode. |
-| monitor-interval            | connector.iceberg.monitor-interval            | N/A                          | 60s                              | Monitor interval to discover splits from new snapshots. Applicable only for streaming read. |
-| include-column-stats        | connector.iceberg.include-column-stats        | N/A                          | false                            | Create a new scan from this that loads the column stats with each data file. Column stats include: value count, null value count, lower bounds, and upper bounds. |
-| max-planning-snapshot-count | connector.iceberg.max-planning-snapshot-count | N/A                          | Integer.MAX_VALUE                | Max number of snapshots limited per split enumeration. Applicable only to streaming read. |
-| limit                       | connector.iceberg.limit                       | N/A                          | -1                               | Limited output number of rows.                               |
+| Read option                   | Flink configuration                             | Table property               | Default                          | Description                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                        |
+|-------------------------------|-------------------------------------------------|------------------------------|----------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| snapshot-id                   | N/A                                             | N/A                          | null                             | For time travel in batch mode. Read data from the specified snapshot-id.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |
+| case-sensitive                | connector.iceberg.case-sensitive                | N/A                          | false                            | If true, match column name in a case sensitive way.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                |
+| as-of-timestamp               | N/A                                             | N/A                          | null                             | For time travel in batch mode. Read data from the most recent snapshot as of the given time in milliseconds.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                       |
+| starting-strategy             | connector.iceberg.starting-strategy             | N/A                          | INCREMENTAL_FROM_LATEST_SNAPSHOT | Starting strategy for streaming execution. TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode. The incremental mode starts from the current snapshot exclusive. INCREMENTAL_FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive. INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. If the timestamp is between two snapshots, it should start from the snapshot after the timestamp. Just for FIP27 Source. |
+| start-snapshot-timestamp      | N/A                                             | N/A                          | null                             | Start to read data from the most recent snapshot as of the given time in milliseconds.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                             |
+| start-snapshot-id             | N/A                                             | N/A                          | null                             | Start to read data from the specified snapshot-id.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                 |
+| end-snapshot-id               | N/A                                             | N/A                          | The latest snapshot id           | Specifies the end snapshot.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                        |
+| branch                        | N/A                                             | N/A                          | main                             | Specifies the branch to read from in batch mode                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                    |
+| tag                           | N/A                                             | N/A                          | null                             | Specifies the tag to read from in batch mode                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                       |
+| start-tag                     | N/A                                             | N/A                          | null                             | Specifies the starting tag to read from for incremental reads                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                      |
+| end-tag                       | N/A                                             | N/A                          | null                             | Specifies the ending tag to to read from for incremental reads                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                     |
+| split-size                    | connector.iceberg.split-size                    | read.split.target-size       | 128 MB                           | Target size when combining input splits.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |
+| split-lookback                | connector.iceberg.split-file-open-cost          | read.split.planning-lookback | 10                               | Number of bins to consider when combining input splits.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            |
+| split-file-open-cost          | connector.iceberg.split-file-open-cost          | read.split.open-file-cost    | 4MB                              | The estimated cost to open a file, used as a minimum weight when combining splits.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                 |
+| streaming                     | connector.iceberg.streaming                     | N/A                          | false                            | Sets whether the current task runs in streaming or batch mode.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                     |
+| monitor-interval              | connector.iceberg.monitor-interval              | N/A                          | 60s                              | Monitor interval to discover splits from new snapshots. Applicable only for streaming read.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                        |
+| include-column-stats          | connector.iceberg.include-column-stats          | N/A                          | false                            | Create a new scan from this that loads the column stats with each data file. Column stats include: value count, null value count, lower bounds, and upper bounds.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |
+| max-planning-snapshot-count   | connector.iceberg.max-planning-snapshot-count   | N/A                          | Integer.MAX_VALUE                | Max number of snapshots limited per split enumeration. Applicable only to streaming read.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                          |
+| limit                         | connector.iceberg.limit                         | N/A                          | -1                               | Limited output number of rows.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                     |
+| max-allowed-planning-failures | connector.iceberg.max-allowed-planning-failures | N/A                          | 3                                | Max allowed consecutive failures for scan planning before failing the job. Set to -1 for never failing the job for scan planing failure.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |
 
 
 ### Write options
@@ -162,4 +164,4 @@
 | compression-codec      | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write      |
 | compression-level      | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write |
 | compression-strategy   | Table write.orc.compression-strategy       | Overrides this table's compression strategy for ORC tables for this write |
-| write-parallelism      | Upstream operator parallelism              | Overrides the writer parallelism                             |
\ No newline at end of file
+| write-parallelism      | Upstream operator parallelism              | Overrides the writer parallelism                             |
diff --git a/docs/content/flink-connector.md b/docs/content/flink-connector.md
index dc1520a..aaf27ff 100644
--- a/docs/content/flink-connector.md
+++ b/docs/content/flink-connector.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_connector
         weight: 200
 ---
 <!--
diff --git a/docs/content/flink-ddl.md b/docs/content/flink-ddl.md
index 67f9e21..f0a484a 100644
--- a/docs/content/flink-ddl.md
+++ b/docs/content/flink-ddl.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_ddl
         weight: 200
 ---
 <!--
@@ -149,8 +150,8 @@
 ```sql
 CREATE TABLE `hive_catalog`.`default`.`sample` (
     id BIGINT COMMENT 'unique id',
-    data STRING
-);
+    data STRING NOT NULL
+) WITH ('format-version'='2');
 ```
 
 Table create commands support the commonly used [Flink create clauses](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/create/) including:
@@ -159,20 +160,35 @@
 * `COMMENT 'table document'` to set a table description.
 * `WITH ('key'='value', ...)` to set [table configuration](../configuration) which will be stored in Iceberg table properties.
 
-Currently, it does not support computed column, primary key and watermark definition etc.
+Currently, it does not support computed column and watermark definition etc.
 
-### `PARTITIONED BY`
+#### `PRIMARY KEY`
+
+Primary key constraint can be declared for a column or a set of columns, which must be unique and do not contain null.
+It's required for [`UPSERT` mode](../flink-writes/#upsert).
+
+```sql
+CREATE TABLE `hive_catalog`.`default`.`sample` (
+    id BIGINT COMMENT 'unique id',
+    data STRING NOT NULL,
+    PRIMARY KEY(`id`) NOT ENFORCED
+) WITH ('format-version'='2');
+```
+
+#### `PARTITIONED BY`
 
 To create a partition table, use `PARTITIONED BY`:
 
 ```sql
 CREATE TABLE `hive_catalog`.`default`.`sample` (
-                                                   id BIGINT COMMENT 'unique id',
-                                                   data STRING
-) PARTITIONED BY (data);
+    id BIGINT COMMENT 'unique id',
+    data STRING NOT NULL
+) 
+PARTITIONED BY (data) 
+WITH ('format-version'='2');
 ```
 
-Iceberg support hidden partition but Flink don't support partitioning by a function on columns, so there is no way to support hidden partition in Flink DDL.
+Iceberg supports hidden partitioning but Flink doesn't support partitioning by a function on columns. There is no way to support hidden partitions in the Flink DDL.
 
 ### `CREATE TABLE LIKE`
 
@@ -180,8 +196,8 @@
 
 ```sql
 CREATE TABLE `hive_catalog`.`default`.`sample` (
-                                                   id BIGINT COMMENT 'unique id',
-                                                   data STRING
+    id BIGINT COMMENT 'unique id',
+    data STRING
 );
 
 CREATE TABLE  `hive_catalog`.`default`.`sample_like` LIKE `hive_catalog`.`default`.`sample`;
diff --git a/docs/content/flink-getting-started.md b/docs/content/flink-getting-started.md
index 078bc12..c5dbf59 100644
--- a/docs/content/flink-getting-started.md
+++ b/docs/content/flink-getting-started.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_getting_started
         weight: 100
 ---
 <!--
@@ -53,7 +54,7 @@
 Download Flink from the [Apache download page](https://flink.apache.org/downloads.html). Iceberg uses Scala 2.12 when compiling the Apache `iceberg-flink-runtime` jar, so it's recommended to use Flink 1.16 bundled with Scala 2.12.
 
 ```bash
-FLINK_VERSION=1.16.1
+FLINK_VERSION=1.16.2
 SCALA_VERSION=2.12
 APACHE_FLINK_URL=https://archive.apache.org/dist/flink/
 wget ${APACHE_FLINK_URL}/flink-${FLINK_VERSION}/flink-${FLINK_VERSION}-bin-scala_${SCALA_VERSION}.tgz
@@ -82,10 +83,15 @@
 # HADOOP_HOME is your hadoop root directory after unpack the binary package.
 export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath`   
 
-./bin/sql-client.sh embedded -j <flink-runtime-directory>/iceberg-flink-runtime-1.16-{{% icebergVersion %}}.jar shell
+# Below works for 1.15 or less
+./bin/sql-client.sh embedded -j <flink-runtime-directory>/iceberg-flink-runtime-1.15-{{% icebergVersion %}}.jar shell
+
+# 1.16 or above has a regression in loading external jar via -j option. See FLINK-30035 for details.
+put iceberg-flink-runtime-1.16-{{% icebergVersion %}}.jar in flink/lib dir
+./bin/sql-client.sh embedded shell
 ```
 
-By default, Iceberg ships with Hadoop jars for Hadoop catalog. To use Hive catalog, load the Hive jars when opening the Flink SQL client. Fortunately, Flink has provided a [bundled hive jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.9_2.12/1.16.1/flink-sql-connector-hive-2.3.9_2.12-1.16.1.jar) for the SQL client. An example on how to download the dependencies and get started:
+By default, Iceberg ships with Hadoop jars for Hadoop catalog. To use Hive catalog, load the Hive jars when opening the Flink SQL client. Fortunately, Flink has provided a [bundled hive jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.9_2.12/1.16.2/flink-sql-connector-hive-2.3.9_2.12-1.16.2.jar) for the SQL client. An example on how to download the dependencies and get started:
 
 ```bash
 # HADOOP_HOME is your hadoop root directory after unpack the binary package.
@@ -99,7 +105,7 @@
 
 HIVE_VERSION=2.3.9
 SCALA_VERSION=2.12
-FLINK_VERSION=1.16.1
+FLINK_VERSION=1.16.2
 FLINK_CONNECTOR_URL=${MAVEN_URL}/org/apache/flink
 FLINK_CONNECTOR_PACKAGE=flink-sql-connector-hive
 wget ${FLINK_CONNECTOR_URL}/${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_VERSION}/${FLINK_VERSION}/${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_VERSION}-${FLINK_VERSION}.jar
@@ -116,7 +122,7 @@
 Install the Apache Flink dependency using `pip`:
 
 ```python
-pip install apache-flink==1.16.1
+pip install apache-flink==1.16.2
 ```
 
 Provide a `file://` path to the `iceberg-flink-runtime` jar, which can be obtained by building the project and looking at `<iceberg-root-dir>/flink-runtime/build/libs`, or downloading it from the [Apache official repository](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime/). Third-party jars can be added to `pyflink` via:
@@ -273,7 +279,7 @@
 
 ### Branch Writes
 Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink`
-For more information on branches please refer to [branches](../../tables/branching).
+For more information on branches please refer to [branches](../tables/branching).
 ```java
 FlinkSink.forRowData(input)
     .tableLoader(tableLoader)
diff --git a/docs/content/flink-queries.md b/docs/content/flink-queries.md
index 2d62d18..fa17fdb 100644
--- a/docs/content/flink-queries.md
+++ b/docs/content/flink-queries.md
@@ -6,6 +6,7 @@
 menu:
    main:
       parent: Flink
+      identifier: flink_queries
       weight: 300
 ---
 <!--
@@ -194,7 +195,7 @@
     .streaming(true)
     .streamingStartingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT)
     .monitorInterval(Duration.ofSeconds(60))
-    .build()
+    .build();
 
 DataStream<RowData> stream = env.fromSource(
     source,
@@ -427,15 +428,15 @@
 SELECT * FROM prod.db.table$partitions;
 ```
 
-| partition      | record_count | file_count | spec_id |
-| -------------- | ------------ | ---------- | ------- |
-| {20211001, 11} | 1            | 1          | 0       |
-| {20211002, 11} | 1            | 1          | 0       |
-| {20211001, 10} | 1            | 1          | 0       |
-| {20211002, 10} | 1            | 1          | 0       |
+| partition      | spec_id | record_count  | file_count | total_data_file_size_in_bytes | position_delete_record_count | position_delete_file_count | equality_delete_record_count | equality_delete_file_count | last_updated_at(μs) | last_updated_snapshot_id |
+| -------------- |---------|---------------|------------|--------------------------|------------------------------|----------------------------|------------------------------|----------------------------|---------------------|--------------------------|
+| {20211001, 11} | 0       | 1             | 1          | 100                      | 2                            | 1                          | 0                            | 0                          | 1633086034192000    | 9205185327307503337      |
+| {20211002, 11} | 0       | 4             | 3          | 500                      | 1                            | 1                          | 0                            | 0                          | 1633172537358000    | 867027598972211003       |
+| {20211001, 10} | 0       | 7             | 4          | 700                      | 0                            | 0                          | 0                            | 0                          | 1633082598716000    | 3280122546965981531      |
+| {20211002, 10} | 0       | 3             | 2          | 400                      | 0                            | 0                          | 1                            | 1                          | 1633169159489000    | 6941468797545315876      |
 
 Note:
-For unpartitioned tables, the partitions table will contain only the record_count and file_count columns.
+For unpartitioned tables, the partitions table will not contain the partition and spec_id fields.
 
 ### All Metadata Tables
 
diff --git a/docs/content/flink-writes.md b/docs/content/flink-writes.md
index f61416b..690fcd8 100644
--- a/docs/content/flink-writes.md
+++ b/docs/content/flink-writes.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Flink
+        identifier: flink_writes
         weight: 400
 ---
 <!--
@@ -68,9 +69,9 @@
 
 ```sql
 CREATE TABLE `hive_catalog`.`default`.`sample` (
-  `id`  INT UNIQUE COMMENT 'unique id',
-  `data` STRING NOT NULL,
- PRIMARY KEY(`id`) NOT ENFORCED
+    `id`  INT UNIQUE COMMENT 'unique id',
+    `data` STRING NOT NULL,
+    PRIMARY KEY(`id`) NOT ENFORCED
 ) with ('format-version'='2', 'write.upsert.enabled'='true');
 ```
 
@@ -192,7 +193,7 @@
 
 ### Branch Writes
 Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink`
-For more information on branches please refer to [branches](../../tables/branching).
+For more information on branches please refer to [branches](../tables/branching).
 ```java
 FlinkSink.forRowData(input)
     .tableLoader(tableLoader)
diff --git a/docs/content/hive-migration.md b/docs/content/hive-migration.md
index c4ef256..21d38d7 100644
--- a/docs/content/hive-migration.md
+++ b/docs/content/hive-migration.md
@@ -4,6 +4,7 @@
 menu:
   main:
     parent: "Migration"
+    identifier: hive_migration
     weight: 200
 ---
 <!--
diff --git a/docs/content/hive.md b/docs/content/hive.md
index d113857..adb79e6 100644
--- a/docs/content/hive.md
+++ b/docs/content/hive.md
@@ -27,6 +27,7 @@
 a [StorageHandler](https://cwiki.apache.org/confluence/display/Hive/StorageHandlers).
 
 ## Feature support
+
 Iceberg compatibility with Hive 2.x and Hive 3.1.2/3 supports the following features:
 
 * Creating a table
@@ -38,8 +39,7 @@
 DML operations work only with MapReduce execution engine.
 {{< /hint >}}
 
-With Hive version 4.0.0-alpha-2 and above,
-the Iceberg integration when using HiveCatalog supports the following additional features:
+With Hive version 4.0.0-alpha-2 and above, Iceberg integration when using HiveCatalog supports the following additional features:
 
 * Altering a table with expiring snapshots.
 * Create a table like an existing table (CTLT table)
@@ -48,8 +48,7 @@
 * Supporting table rollback
 * Honours sort orders on existing tables when writing a table [Sort orders specification](https://iceberg.apache.org/spec/#sort-orders)
 
-With Hive version 4.0.0-alpha-1 and above,
-the Iceberg integration when using HiveCatalog supports the following additional features:
+With Hive version 4.0.0-alpha-1 and above, Iceberg integration when using HiveCatalog supports the following additional features:
 
 * Creating an Iceberg identity-partitioned table
 * Creating an Iceberg table with any partition spec, including the various transforms supported by Iceberg
@@ -71,9 +70,19 @@
 
 ## Enabling Iceberg support in Hive
 
+Hive 4 comes with `hive-iceberg` that ships Iceberg, so no additional downloads or jars are needed. For older versions of Hive a runtime jar has to be added.
+
+### Hive 4.0.0-beta-1
+
+Hive 4.0.0-beta-1 comes with the Iceberg 1.3.0 included.
+
+### Hive 4.0.0-alpha-2
+
+Hive 4.0.0-alpha-2 comes with the Iceberg 0.14.1 included.
+
 ### Hive 4.0.0-alpha-1
 
-Hive 4.0.0-alpha-1 comes with the Iceberg 0.13.1 included. No additional downloads or jars are needed.
+Hive 4.0.0-alpha-1 comes with the Iceberg 0.13.1 included.
 
 ### Hive 2.3.x, Hive 3.1.x
 
diff --git a/docs/content/java-api-quickstart.md b/docs/content/java-api-quickstart.md
index 24b61b4..358ed2b 100644
--- a/docs/content/java-api-quickstart.md
+++ b/docs/content/java-api-quickstart.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: "API"
+        identifier: java_api_quickstart
         weight: 100
 ---
 <!--
diff --git a/docs/content/java-api.md b/docs/content/java-api.md
index 15731e4..62b51e0 100644
--- a/docs/content/java-api.md
+++ b/docs/content/java-api.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: "API"
+        identifier: java_api
         weight: 200
 ---
 <!--
diff --git a/docs/content/java-custom-catalog.md b/docs/content/java-custom-catalog.md
index 0132dd1..59238a2 100644
--- a/docs/content/java-custom-catalog.md
+++ b/docs/content/java-custom-catalog.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: "API"
+        identifier: java_custom_catalog
         weight: 300
 ---
 <!--
diff --git a/docs/content/jdbc.md b/docs/content/jdbc.md
index c096c29..f7f7004 100644
--- a/docs/content/jdbc.md
+++ b/docs/content/jdbc.md
@@ -4,6 +4,7 @@
 menu:
     main:
         parent: Integrations
+        identifier: jdbc_integration
         weight: 0
 ---
 <!--
diff --git a/docs/content/maintenance.md b/docs/content/maintenance.md
index 07a3099..3e5e2a0 100644
--- a/docs/content/maintenance.md
+++ b/docs/content/maintenance.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_maintenance
         weight: 0
 ---
 <!--
diff --git a/docs/content/metrics-reporting.md b/docs/content/metrics-reporting.md
new file mode 100644
index 0000000..5b87bfe
--- /dev/null
+++ b/docs/content/metrics-reporting.md
@@ -0,0 +1,174 @@
+---
+title: "Metrics Reporting"
+url: metrics-reporting
+aliases:
+    - "tables/metrics-reporting"
+menu:
+    main:
+        parent: Tables
+        identifier: metrics_reporting
+        weight: 0
+---
+<!--
+ - 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.
+ -->
+
+# Metrics Reporting
+
+As of 1.1.0 Iceberg supports the [`MetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReporter.html) and the [`MetricsReport`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReport.html) APIs. These two APIs allow expressing different metrics reports while supporting a pluggable way of reporting these reports.
+
+## Type of Reports
+
+### ScanReport
+A [`ScanReport`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/ScanReport.html) carries metrics being collected during scan planning against a given table. Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like:
+* total scan planning duration
+* number of data/delete files included in the result
+* number of data/delete manifests scanned/skipped
+* number of data/delete files scanned/skipped
+* number of equality/positional delete files scanned
+
+
+### CommitReport
+A [`CommitReport`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/CommitReport.html) carries metrics being collected after committing changes to a table (aka producing a snapshot). Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like:
+* total duration
+* number of attempts required for the commit to succeed
+* number of added/removed data/delete files
+* number of added/removed equality/positional delete files
+* number of added/removed equality/positional deletes
+
+
+## Available Metrics Reporters
+
+### [`LoggingMetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/LoggingMetricsReporter.html)
+
+This is the default metrics reporter when nothing else is configured and its purpose is to log results to the log file. Example output would look as shown below:
+
+```
+INFO org.apache.iceberg.metrics.LoggingMetricsReporter - Received metrics report: 
+ScanReport{
+    tableName=scan-planning-with-eq-and-pos-delete-files, 
+    snapshotId=2, 
+    filter=ref(name="data") == "(hash-27fa7cc0)", 
+    schemaId=0, 
+    projectedFieldIds=[1, 2], 
+    projectedFieldNames=[id, data], 
+    scanMetrics=ScanMetricsResult{
+        totalPlanningDuration=TimerResult{timeUnit=NANOSECONDS, totalDuration=PT0.026569404S, count=1}, 
+        resultDataFiles=CounterResult{unit=COUNT, value=1}, 
+        resultDeleteFiles=CounterResult{unit=COUNT, value=2}, 
+        totalDataManifests=CounterResult{unit=COUNT, value=1}, 
+        totalDeleteManifests=CounterResult{unit=COUNT, value=1}, 
+        scannedDataManifests=CounterResult{unit=COUNT, value=1}, 
+        skippedDataManifests=CounterResult{unit=COUNT, value=0}, 
+        totalFileSizeInBytes=CounterResult{unit=BYTES, value=10}, 
+        totalDeleteFileSizeInBytes=CounterResult{unit=BYTES, value=20}, 
+        skippedDataFiles=CounterResult{unit=COUNT, value=0}, 
+        skippedDeleteFiles=CounterResult{unit=COUNT, value=0}, 
+        scannedDeleteManifests=CounterResult{unit=COUNT, value=1}, 
+        skippedDeleteManifests=CounterResult{unit=COUNT, value=0}, 
+        indexedDeleteFiles=CounterResult{unit=COUNT, value=2}, 
+        equalityDeleteFiles=CounterResult{unit=COUNT, value=1}, 
+        positionalDeleteFiles=CounterResult{unit=COUNT, value=1}}, 
+    metadata={
+        iceberg-version=Apache Iceberg 1.4.0-SNAPSHOT (commit 4868d2823004c8c256a50ea7c25cff94314cc135)}}
+```
+
+```
+INFO org.apache.iceberg.metrics.LoggingMetricsReporter - Received metrics report: 
+CommitReport{
+    tableName=scan-planning-with-eq-and-pos-delete-files, 
+    snapshotId=1, 
+    sequenceNumber=1, 
+    operation=append, 
+    commitMetrics=CommitMetricsResult{
+        totalDuration=TimerResult{timeUnit=NANOSECONDS, totalDuration=PT0.098429626S, count=1}, 
+        attempts=CounterResult{unit=COUNT, value=1}, 
+        addedDataFiles=CounterResult{unit=COUNT, value=1}, 
+        removedDataFiles=null, 
+        totalDataFiles=CounterResult{unit=COUNT, value=1}, 
+        addedDeleteFiles=null, 
+        addedEqualityDeleteFiles=null, 
+        addedPositionalDeleteFiles=null, 
+        removedDeleteFiles=null, 
+        removedEqualityDeleteFiles=null, 
+        removedPositionalDeleteFiles=null, 
+        totalDeleteFiles=CounterResult{unit=COUNT, value=0}, 
+        addedRecords=CounterResult{unit=COUNT, value=1}, 
+        removedRecords=null, 
+        totalRecords=CounterResult{unit=COUNT, value=1}, 
+        addedFilesSizeInBytes=CounterResult{unit=BYTES, value=10}, 
+        removedFilesSizeInBytes=null, 
+        totalFilesSizeInBytes=CounterResult{unit=BYTES, value=10}, 
+        addedPositionalDeletes=null, 
+        removedPositionalDeletes=null, 
+        totalPositionalDeletes=CounterResult{unit=COUNT, value=0}, 
+        addedEqualityDeletes=null, 
+        removedEqualityDeletes=null, 
+        totalEqualityDeletes=CounterResult{unit=COUNT, value=0}}, 
+    metadata={
+        iceberg-version=Apache Iceberg 1.4.0-SNAPSHOT (commit 4868d2823004c8c256a50ea7c25cff94314cc135)}}
+```
+
+
+### [`RESTMetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/rest/RESTMetricsReporter.html)
+
+This is the default when using the [`RESTCatalog`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/rest/RESTCatalog.html) and its purpose is to send metrics to a REST server at the `/v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics` endpoint as defined in the [REST OpenAPI spec](https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml).
+
+Sending metrics via REST can be controlled with the `rest-metrics-reporting-enabled` (defaults to `true`) property.
+
+
+## Implementing a custom Metrics Reporter
+
+Implementing the [`MetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReporter.html) API gives full flexibility in dealing with incoming [`MetricsReport`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReport.html) instances. For example, it would be possible to send results to a Prometheus endpoint or any other observability framework/system.
+
+Below is a short example illustrating an `InMemoryMetricsReporter` that stores reports in a list and makes them available:
+```java
+public class InMemoryMetricsReporter implements MetricsReporter {
+
+  private List<MetricsReport> metricsReports = Lists.newArrayList();
+
+  @Override
+  public void report(MetricsReport report) {
+    metricsReports.add(report);
+  }
+
+  public List<MetricsReport> reports() {
+    return metricsReports;
+  }
+}
+```
+
+## Registering a custom Metrics Reporter
+
+### Via Catalog Configuration
+
+The [catalog property](../configuration#catalog-properties) `metrics-reporter-impl` allows registering a given [`MetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReporter.html) by specifying its fully-qualified class name, e.g. `metrics-reporter-impl=org.apache.iceberg.metrics.InMemoryMetricsReporter`.
+
+### Via the Java API during Scan planning
+
+Independently of the [`MetricsReporter`](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/metrics/MetricsReporter.html) being registered at the catalog level via the `metrics-reporter-impl` property, it is also possible to supply additional reporters during scan planning as shown below:
+
+```java
+TableScan tableScan = 
+    table
+        .newScan()
+        .metricsReporter(customReporterOne)
+        .metricsReporter(customReporterTwo);
+
+try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) {
+  // ...
+}
+```
\ No newline at end of file
diff --git a/docs/content/nessie.md b/docs/content/nessie.md
index b64847f..47b91c8 100644
--- a/docs/content/nessie.md
+++ b/docs/content/nessie.md
@@ -4,6 +4,7 @@
 menu:
     main:
         parent: Integrations
+        identifier: nessie_integration
         weight: 0
 ---
 <!--
@@ -43,7 +44,7 @@
 
 ## Spark SQL Extensions
 
-From Spark 3.1 and above, Nessie SQL extensions can be used to manage the Nessie repo as shown below.
+Nessie SQL extensions can be used to manage the Nessie repo as shown below.
 Example for Spark 3.3 with scala 2.12:
 
 ```
@@ -109,8 +110,9 @@
 settings for the `catalog-impl` and the required config to start Nessie correctly.
 Once you have a Nessie catalog you have access to your entire Nessie repo. You can then perform create/delete/merge
 operations on branches and perform commits on branches. Each Iceberg table in a Nessie Catalog is identified by an
-arbitrary length namespace and table name (eg `data.base.name.table`). These namespaces are implicit and don't need to
-be created separately. Any transaction on a Nessie enabled Iceberg table is a single commit in Nessie. Nessie commits
+arbitrary length namespace and table name (eg `data.base.name.table`). These namespaces must be explicitly created 
+as mentioned [here](https://projectnessie.org/blog/namespace-enforcement/).
+Any transaction on a Nessie enabled Iceberg table is a single commit in Nessie. Nessie commits
 can encompass an arbitrary number of actions on an arbitrary number of tables, however in Iceberg this will be limited
 to the set of single table transactions currently available.
 
diff --git a/docs/content/partitioning.md b/docs/content/partitioning.md
index 349d8fa..799fc4f 100644
--- a/docs/content/partitioning.md
+++ b/docs/content/partitioning.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_partitioning
         weight: 0
 ---
 <!--
diff --git a/docs/content/performance.md b/docs/content/performance.md
index 9193987..f5e87e3 100644
--- a/docs/content/performance.md
+++ b/docs/content/performance.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_performance
         weight: 0
 ---
 <!--
diff --git a/docs/content/reliability.md b/docs/content/reliability.md
index 6a11516..9a47668 100644
--- a/docs/content/reliability.md
+++ b/docs/content/reliability.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_reliability
         weight: 0
 ---
 <!--
diff --git a/docs/content/schemas.md b/docs/content/schemas.md
index 502d330..03805fa 100644
--- a/docs/content/schemas.md
+++ b/docs/content/schemas.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Tables
+        identifier: tables_schema
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-configuration.md b/docs/content/spark-configuration.md
index 926ec02..f94efdc 100644
--- a/docs/content/spark-configuration.md
+++ b/docs/content/spark-configuration.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_configuration
         weight: 0
 ---
 <!--
@@ -40,6 +41,14 @@
 # omit uri to use the same URI as Spark: hive.metastore.uris in hive-site.xml
 ```
 
+Below is an example for a REST catalog named `rest_prod` that loads tables from REST URL `http://localhost:8080`:
+
+```plain
+spark.sql.catalog.rest_prod = org.apache.iceberg.spark.SparkCatalog
+spark.sql.catalog.rest_prod.type = rest
+spark.sql.catalog.rest_prod.uri = http://localhost:8080
+```
+
 Iceberg also supports a directory-based catalog in HDFS that can be configured using `type=hadoop`:
 
 ```plain
@@ -66,12 +75,16 @@
 | Property                                           | Values                        | Description                                                          |
 | -------------------------------------------------- | ----------------------------- | -------------------------------------------------------------------- |
 | spark.sql.catalog._catalog-name_.type              | `hive`, `hadoop` or `rest`    | The underlying Iceberg catalog implementation, `HiveCatalog`, `HadoopCatalog`, `RESTCatalog` or left unset if using a custom catalog |
-| spark.sql.catalog._catalog-name_.catalog-impl      |                               | The underlying Iceberg catalog implementation.|
+| spark.sql.catalog._catalog-name_.catalog-impl      |                               | The custom Iceberg catalog implementation. If `type` is null, `catalog-impl` must not be null. |
+| spark.sql.catalog._catalog-name_.io-impl                      |                               | The custom FileIO implementation. |
+| spark.sql.catalog._catalog-name_.metrics-reporter-impl        |                               | The custom MetricsReporter implementation.  |
 | spark.sql.catalog._catalog-name_.default-namespace | default                       | The default current namespace for the catalog |
-| spark.sql.catalog._catalog-name_.uri               | thrift://host:port            | Metastore connect URI; default from `hive-site.xml` |
+| spark.sql.catalog._catalog-name_.uri               | thrift://host:port            | Hive metastore URL for hive typed catalog, REST URL for REST typed catalog |
 | spark.sql.catalog._catalog-name_.warehouse         | hdfs://nn:8020/warehouse/path | Base path for the warehouse directory |
 | spark.sql.catalog._catalog-name_.cache-enabled     | `true` or `false`             | Whether to enable catalog cache, default value is `true` |
-| spark.sql.catalog._catalog-name_.cache.expiration-interval-ms | `30000` (30 seconds) | Duration after which cached catalog entries are expired; Only effective if `cache-enabled` is `true`. `-1` disables cache expiration and `0` disables caching entirely, irrespective of `cache-enabled`. Default is `30000` (30 seconds) |                                                   |
+| spark.sql.catalog._catalog-name_.cache.expiration-interval-ms | `30000` (30 seconds) | Duration after which cached catalog entries are expired; Only effective if `cache-enabled` is `true`. `-1` disables cache expiration and `0` disables caching entirely, irrespective of `cache-enabled`. Default is `30000` (30 seconds) |
+| spark.sql.catalog._catalog-name_.table-default._propertyKey_  |                               | Default Iceberg table property value for property key _propertyKey_, which will be set on tables created by this catalog if not overridden                                                                                               |
+| spark.sql.catalog._catalog-name_.table-override._propertyKey_ |                               | Enforced Iceberg table property value for property key _propertyKey_, which cannot be overridden by user                                                                                                                                 |
 
 Additional properties can be found in common [catalog configuration](../configuration#catalog-properties).
 
@@ -176,8 +189,26 @@
 | write-format           | Table write.format.default | File format to use for this write operation; parquet, avro, or orc |
 | target-file-size-bytes | As per table property      | Overrides this table's write.target-file-size-bytes          |
 | check-nullability      | true                       | Sets the nullable check on fields                            |
-| snapshot-property._custom-key_    | null            | Adds an entry with custom-key and corresponding value in the snapshot summary  |
+| snapshot-property._custom-key_    | null            | Adds an entry with custom-key and corresponding value in the snapshot summary (the `snapshot-property.` prefix is only required for DSv2)  |
 | fanout-enabled       | false        | Overrides this table's write.spark.fanout.enabled  |
 | check-ordering       | true        | Checks if input schema and table schema are same  |
 | isolation-level | null | Desired isolation level for Dataframe overwrite operations.  `null` => no checks (for idempotent writes), `serializable` => check for concurrent inserts or deletes in destination partitions, `snapshot` => checks for concurrent deletes in destination partitions. |
 | validate-from-snapshot-id | null | If isolation level is set, id of base snapshot from which to check concurrent write conflicts into a table. Should be the snapshot before any reads from the table. Can be obtained via [Table API](../../api#table-metadata) or [Snapshots table](../spark-queries#snapshots). If null, the table's oldest known snapshot is used. |
+| compression-codec      | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write      |
+| compression-level      | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write |
+| compression-strategy   | Table write.orc.compression-strategy       | Overrides this table's compression strategy for ORC tables for this write |
+
+CommitMetadata provides an interface to add custom metadata to a snapshot summary during a SQL execution, which can be beneficial for purposes such as auditing or change tracking. If properties start with `snapshot-property.`, then that prefix will be removed from each property. Here is an example:
+
+```java
+import org.apache.iceberg.spark.CommitMetadata;
+
+Map<String, String> properties = Maps.newHashMap();
+properties.put("property_key", "property_value");
+CommitMetadata.withCommitProperties(properties,
+        () -> {
+            spark.sql("DELETE FROM " + tableName + " where id = 1");
+            return 0;
+        },
+        RuntimeException.class);
+```
\ No newline at end of file
diff --git a/docs/content/spark-ddl.md b/docs/content/spark-ddl.md
index b862285..77684b9 100644
--- a/docs/content/spark-ddl.md
+++ b/docs/content/spark-ddl.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_ddl
         weight: 0
 ---
 <!--
@@ -78,15 +79,17 @@
 
 Supported transformations are:
 
-* `years(ts)`: partition by year
-* `months(ts)`: partition by month
-* `days(ts)` or `date(ts)`: equivalent to dateint partitioning
-* `hours(ts)` or `date_hour(ts)`: equivalent to dateint and hour partitioning
+* `year(ts)`: partition by year
+* `month(ts)`: partition by month
+* `day(ts)` or `date(ts)`: equivalent to dateint partitioning
+* `hour(ts)` or `date_hour(ts)`: equivalent to dateint and hour partitioning
 * `bucket(N, col)`: partition by hashed value mod N buckets
 * `truncate(L, col)`: partition by value truncated to L
     * Strings are truncated to the given length
     * Integers and longs truncate to bins: `truncate(10, i)` produces partitions 0, 10, 20, 30, ...
 
+Note: Old syntax of `years(ts)`, `months(ts)`, `days(ts)` and `hours(ts)` are also supported for compatibility. 
+
 ## `CREATE TABLE ... AS SELECT`
 
 Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](../spark-configuration#catalog-configuration). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](../spark-configuration#replacing-the-session-catalog).
@@ -305,13 +308,18 @@
 ALTER TABLE prod.db.sample ALTER COLUMN nested.col AFTER other_col
 ```
 
-Nullability can be changed using `SET NOT NULL` and `DROP NOT NULL`:
+Nullability for a non-nullable column can be changed using `DROP NOT NULL`:
 
 ```sql
 ALTER TABLE prod.db.sample ALTER COLUMN id DROP NOT NULL
 ```
 
 {{< hint info >}}
+It is not possible to change a nullable column to a non-nullable column with `SET NOT NULL` because Iceberg doesn't know whether there is existing data with null values.
+{{< /hint >}}
+
+
+{{< hint info >}}
 `ALTER COLUMN` is not used to update `struct` types. Use `ADD COLUMN` and `DROP COLUMN` to add or remove struct fields.
 {{< /hint >}}
 
@@ -342,7 +350,7 @@
 ```sql
 ALTER TABLE prod.db.sample ADD PARTITION FIELD bucket(16, id)
 ALTER TABLE prod.db.sample ADD PARTITION FIELD truncate(4, data)
-ALTER TABLE prod.db.sample ADD PARTITION FIELD years(ts)
+ALTER TABLE prod.db.sample ADD PARTITION FIELD year(ts)
 -- use optional AS keyword to specify a custom name for the partition field 
 ALTER TABLE prod.db.sample ADD PARTITION FIELD bucket(16, id) AS shard
 ```
@@ -368,7 +376,7 @@
 ALTER TABLE prod.db.sample DROP PARTITION FIELD catalog
 ALTER TABLE prod.db.sample DROP PARTITION FIELD bucket(16, id)
 ALTER TABLE prod.db.sample DROP PARTITION FIELD truncate(4, data)
-ALTER TABLE prod.db.sample DROP PARTITION FIELD years(ts)
+ALTER TABLE prod.db.sample DROP PARTITION FIELD year(ts)
 ALTER TABLE prod.db.sample DROP PARTITION FIELD shard
 ```
 
@@ -390,9 +398,9 @@
 A partition field can be replaced by a new partition field in a single metadata update by using `REPLACE PARTITION FIELD`:
 
 ```sql
-ALTER TABLE prod.db.sample REPLACE PARTITION FIELD ts_day WITH days(ts)
+ALTER TABLE prod.db.sample REPLACE PARTITION FIELD ts_day WITH day(ts)
 -- use optional AS keyword to specify a custom name for the new partition field 
-ALTER TABLE prod.db.sample REPLACE PARTITION FIELD ts_day WITH days(ts) AS day_of_ts
+ALTER TABLE prod.db.sample REPLACE PARTITION FIELD ts_day WITH day(ts) AS day_of_ts
 ```
 
 ### `ALTER TABLE ... WRITE ORDERED BY`
@@ -471,43 +479,77 @@
 
 #### `ALTER TABLE ... CREATE BRANCH`
 
-Branches can be created via the `CREATE BRANCH` statement, which includes 
-the snapshot to create the branch at and an optional retention clause.
+Branches can be created via the `CREATE BRANCH` statement with the following options:
+* Do not fail if the branch already exists with `IF NOT EXISTS`
+* Update the branch if it already exists with `CREATE OR REPLACE`
+* Create at a snapshot
+* Create with retention
 
 ```sql
+-- CREATE audit-branch at current snapshot with default retention.
+ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch`
+
+-- CREATE audit-branch at current snapshot with default retention if it doesn't exist.
+ALTER TABLE prod.db.sample CREATE BRANCH IF NOT EXISTS `audit-branch`
+
+-- CREATE audit-branch at current snapshot with default retention or REPLACE it if it already exists.
+ALTER TABLE prod.db.sample CREATE OR REPLACE BRANCH `audit-branch`
+
 -- CREATE audit-branch at snapshot 1234 with default retention.
-ALTER TABLE prod.db.sample CREATE BRANCH audit-branch
+ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch`
 AS OF VERSION 1234
 
--- CREATE audit-branch at snapshot 1234, retain audit-branch for 31 days, and retain the latest 31 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots 
-ALTER TABLE prod.db.sample CREATE BRANCH audit-branch
+-- CREATE audit-branch at snapshot 1234, retain audit-branch for 31 days, and retain the latest 31 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots. 
+ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch`
 AS OF VERSION 1234 RETAIN 30 DAYS 
-WITH RETENTION 3 SNAPSHOTS 2 DAYS
+WITH SNAPSHOT RETENTION 3 SNAPSHOTS 2 DAYS
 ```
 
-
 #### `ALTER TABLE ... CREATE TAG`
 
-Tags can be created via the `CREATE TAG` statement, which includes 
-the snapshot to create the branch at and an optional retention clause.
+Tags can be created via the `CREATE TAG` statement with the following options:
+* Do not fail if the tag already exists with `IF NOT EXISTS`
+* Update the tag if it already exists with `CREATE OR REPLACE`
+* Create at a snapshot
+* Create with retention
 
 ```sql
+-- CREATE historical-tag at current snapshot with default retention.
+ALTER TABLE prod.db.sample CREATE TAG `historical-tag`
+
+-- CREATE historical-tag at current snapshot with default retention if it doesn't exist.
+ALTER TABLE prod.db.sample CREATE TAG IF NOT EXISTS `historical-tag`
+
+-- CREATE historical-tag at current snapshot with default retention or REPLACE it if it already exists.
+ALTER TABLE prod.db.sample CREATE OR REPLACE TAG `historical-tag`
+
 -- CREATE historical-tag at snapshot 1234 with default retention.
-ALTER TABLE prod.db.sample CREATE TAG historical-tag AS OF VERSION 1234
+ALTER TABLE prod.db.sample CREATE TAG `historical-tag` AS OF VERSION 1234
 
 -- CREATE historical-tag at snapshot 1234 and retain it for 1 year. 
-ALTER TABLE prod.db.sample CREATE TAG historical-tag 
+ALTER TABLE prod.db.sample CREATE TAG `historical-tag` 
 AS OF VERSION 1234 RETAIN 365 DAYS
 ```
 
-### `ALTER TABLE ... REPLACE BRANCH`
+#### `ALTER TABLE ... REPLACE BRANCH`
 
 The snapshot which a branch references can be updated via
 the `REPLACE BRANCH` sql. Retention can also be updated in this statement. 
 
 ```sql
--- REPLACE audit-branch to reference snapshot 4567 and update the retention to 60 days
-ALTER TABLE prod.db.sample REPLACE BRANCH audit-branch
+-- REPLACE audit-branch to reference snapshot 4567 and update the retention to 60 days.
+ALTER TABLE prod.db.sample REPLACE BRANCH `audit-branch`
+AS OF VERSION 4567 RETAIN 60 DAYS
+```
+
+#### `ALTER TABLE ... REPLACE TAG`
+
+The snapshot which a tag references can be updated via
+the `REPLACE TAG` sql. Retention can also be updated in this statement.
+
+```sql
+-- REPLACE historical-tag to reference snapshot 4567 and update the retention to 60 days.
+ALTER TABLE prod.db.sample REPLACE TAG `historical-tag`
 AS OF VERSION 4567 RETAIN 60 DAYS
 ```
 
@@ -516,7 +558,7 @@
 Branches can be removed via the `DROP BRANCH` sql
 
 ```sql
-ALTER TABLE prod.db.sample DROP BRANCH audit-branch
+ALTER TABLE prod.db.sample DROP BRANCH `audit-branch`
 ```
 
 #### `ALTER TABLE ... DROP TAG`
@@ -524,5 +566,5 @@
 Tags can be removed via the `DROP TAG` sql
 
 ```sql
-ALTER TABLE prod.db.sample DROP TAG historical-tag
+ALTER TABLE prod.db.sample DROP TAG `historical-tag`
 ```
\ No newline at end of file
diff --git a/docs/content/spark-getting-started.md b/docs/content/spark-getting-started.md
index a9bbe83..f72bb7e 100644
--- a/docs/content/spark-getting-started.md
+++ b/docs/content/spark-getting-started.md
@@ -7,6 +7,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_getting_started
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-procedures.md b/docs/content/spark-procedures.md
index 382606d..5303982 100644
--- a/docs/content/spark-procedures.md
+++ b/docs/content/spark-procedures.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_procedures
         weight: 0
 ---
 <!--
@@ -129,7 +130,10 @@
 | Argument Name | Required? | Type | Description |
 |---------------|-----------|------|-------------|
 | `table`       | ✔️  | string | Name of the table to update |
-| `snapshot_id` | ✔️  | long   | Snapshot ID to set as current |
+| `snapshot_id` | | long   | Snapshot ID to set as current |
+| `ref` | | string | Snapshot Referece (branch or tag) to set as current |
+
+Either `snapshot_id` or `ref` must be provided but not both.
 
 #### Output
 
@@ -145,6 +149,11 @@
 CALL catalog_name.system.set_current_snapshot('db.sample', 1)
 ```
 
+Set the current snapshot for `db.sample` to tag `s1`:
+```sql
+CALL catalog_name.system.set_current_snapshot(table => 'db.sample', tag => 's1');
+```
+
 ### `cherrypick_snapshot`
 
 Cherry-picks changes from a snapshot into the current table state.
@@ -183,6 +192,34 @@
 CALL catalog_name.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
 ```
 
+### `fast_forward`
+
+Fast-forward the current snapshot of one branch to the latest snapshot of another.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| `table` | ✔️ | string | Name of the table to update |
+| `branch` | ✔️ | string   | Name of the branch to fast-forward |
+| `to` | ✔️ | string | | Name of the branch to be fast-forwarded to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| `branch_updated` | string | Name of the branch that has been fast-forwarded |
+| `previous_ref` | long | The snapshot ID before applying fast-forward |
+| `updated_ref`  | long | The current snapshot ID after applying fast-forward |
+
+#### Examples
+
+Fast-forward the main branch to the head of `audit-branch`
+```sql
+CALL catalog_name.system.fast_forward('my_table', 'main', 'audit-branch')
+```
+
+
 
 ## Metadata management
 
@@ -209,6 +246,7 @@
 | `snapshot_ids` |   | array of long       | Array of snapshot IDs to expire. |
 
 If `older_than` and `retain_last` are omitted, the table's [expiration properties](../configuration/#table-behavior-properties) will be used.
+Snapshots that are still referenced by branches or tags won't be removed. By default, branches and tags never expire, but their retention policy can be changed with the table property `history.expire.max-ref-age-ms`. The `main` branch never expires.
 
 #### Output
 
@@ -282,11 +320,38 @@
 | `options`     | ️   | map<string, string> | Options to be used for actions|
 | `where`       | ️   | string | predicate as a string used for filtering the files. Note that all files that may contain data matching the filter will be selected for rewriting|
 
+#### Options
 
-See the [`RewriteDataFiles` Javadoc](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/actions/RewriteDataFiles.html#field.summary),
-<br/>  [`BinPackStrategy` Javadoc](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/actions/BinPackStrategy.html#field.summary)
-and <br/> [`SortStrategy` Javadoc](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/actions/SortStrategy.html#field.summary)
-for list of all the supported options for this action.
+##### General Options
+| Name | Default Value | Description |
+|------|---------------|-------------|
+| `max-concurrent-file-group-rewrites` | 5 | Maximum number of file groups to be simultaneously rewritten |
+| `partial-progress.enabled` | false | Enable committing groups of files prior to the entire rewrite completing |
+| `partial-progress.max-commits` | 10 | Maximum amount of commits that this rewrite is allowed to produce if partial progress is enabled |
+| `use-starting-sequence-number` | true | Use the sequence number of the snapshot at compaction start time instead of that of the newly produced snapshot |
+| `rewrite-job-order` | none | Force the rewrite job order based on the value. <ul><li>If rewrite-job-order=bytes-asc, then rewrite the smallest job groups first.</li><li>If rewrite-job-order=bytes-desc, then rewrite the largest job groups first.</li><li>If rewrite-job-order=files-asc, then rewrite the job groups with the least files first.</li><li>If rewrite-job-order=files-desc, then rewrite the job groups with the most files first.</li><li>If rewrite-job-order=none, then rewrite job groups in the order they were planned (no specific ordering).</li></ul> |
+| `target-file-size-bytes` | 536870912 (512 MB, default value of `write.target-file-size-bytes` from [table properties](../configuration/#write-properties)) | Target output file size |
+| `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria |
+| `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria |
+| `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria |
+| `rewrite-all` | false | Force rewriting of all provided files overriding other options |
+| `max-file-group-size-bytes` | 107374182400 (100GB) | Largest amount of data that should be rewritten in a single file group. The entire rewrite operation is broken down into pieces based on partitioning and within partitions based on size into file-groups.  This helps with breaking down the rewriting of very large partitions which may not be rewritable otherwise due to the resource constraints of the cluster. |
+| `delete-file-threshold` | 2147483647 | Minimum number of deletes that needs to be associated with a data file for it to be considered for rewriting |
+
+
+##### Options for sort strategy
+
+| Name | Default Value | Description |
+|------|---------------|-------------|
+| `compression-factor` | 1.0 | The number of shuffle partitions and consequently the number of output files created by the Spark sort is based on the size of the input data files used in this file rewriter. Due to compression, the disk file sizes may not accurately represent the size of files in the output. This parameter lets the user adjust the file size used for estimating actual output data size. A factor greater than 1.0 would generate more files than we would expect based on the on-disk file size. A value less than 1.0 would create fewer files than we would expect based on the on-disk size. |
+| `shuffle-partitions-per-file` | 1 | Number of shuffle partitions to use for each output file. Iceberg will use a custom coalesce operation to stitch these sorted partitions back together into a single sorted file. |
+
+##### Options for sort strategy with zorder sort_order
+
+| Name | Default Value | Description |
+|------|---------------|-------------|
+| `var-length-contribution` | 8 | Number of bytes considered from an input column of a type with variable length (String, Binary) |
+| `max-output-size` | 2147483647 | Amount of bytes interleaved in the ZOrder algorithm |
 
 #### Output
 
@@ -294,6 +359,8 @@
 | ------------|------|-------------|
 | `rewritten_data_files_count` | int | Number of data which were re-written by this command |
 | `added_data_files_count`     | int | Number of new data files which were written by this command |
+| `rewritten_bytes_count`      | long | Number of bytes which were written by this command |
+| `failed_data_files_count`    | int | Number of data files that failed to be rewritten when `partial-progress.enabled` is true |
 
 #### Examples
 
@@ -331,9 +398,6 @@
 
 Data files in manifests are sorted by fields in the partition spec. This procedure runs in parallel using a Spark job.
 
-See the [`RewriteManifests` Javadoc](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/actions/RewriteManifests.html)
-to see more configuration options.
-
 {{< hint info >}}
 This procedure invalidates all cached Spark plans that reference the affected table.
 {{< /hint >}}
@@ -364,6 +428,63 @@
 CALL catalog_name.system.rewrite_manifests('db.sample', false)
 ```
 
+### `rewrite_position_delete_files`
+
+Iceberg can rewrite position delete files, which serves two purposes:
+* Minor Compaction: Compact small position delete files into larger ones.  This reduces the size of metadata stored in manifest files and overhead of opening small delete files.
+* Remove Dangling Deletes: Filter out position delete records that refer to data files that are no longer live.  After rewrite_data_files, position delete records pointing to the rewritten data files are not always marked for removal, and can remain tracked by the table's live snapshot metadata.  This is known as the 'dangling delete' problem.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description                      |
+|---------------|-----------|------|----------------------------------|
+| `table`       | ✔️  | string | Name of the table to update      |
+| `options`     | ️   | map<string, string> | Options to be used for procedure |
+
+Dangling deletes are always filtered out during rewriting.
+
+#### Options
+
+| Name | Default Value | Description |
+|------|---------------|-------------|
+| `max-concurrent-file-group-rewrites` | 5 | Maximum number of file groups to be simultaneously rewritten |
+| `partial-progress.enabled` | false | Enable committing groups of files prior to the entire rewrite completing |
+| `partial-progress.max-commits` | 10 | Maximum amount of commits that this rewrite is allowed to produce if partial progress is enabled |
+| `rewrite-job-order` | none | Force the rewrite job order based on the value. <ul><li>If rewrite-job-order=bytes-asc, then rewrite the smallest job groups first.</li><li>If rewrite-job-order=bytes-desc, then rewrite the largest job groups first.</li><li>If rewrite-job-order=files-asc, then rewrite the job groups with the least files first.</li><li>If rewrite-job-order=files-desc, then rewrite the job groups with the most files first.</li><li>If rewrite-job-order=none, then rewrite job groups in the order they were planned (no specific ordering).</li></ul> |
+| `target-file-size-bytes` | 67108864 (64MB, default value of `write.delete.target-file-size-bytes` from [table properties](../configuration/#write-properties)) | Target output file size |
+| `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria |
+| `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria |
+| `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria |
+| `rewrite-all` | false | Force rewriting of all provided files overriding other options |
+| `max-file-group-size-bytes` | 107374182400 (100GB) | Largest amount of data that should be rewritten in a single file group. The entire rewrite operation is broken down into pieces based on partitioning and within partitions based on size into file-groups.  This helps with breaking down the rewriting of very large partitions which may not be rewritable otherwise due to the resource constraints of the cluster. |
+
+#### Output
+
+| Output Name                    | Type | Description                                                                |
+|--------------------------------|------|----------------------------------------------------------------------------|
+| `rewritten_delete_files_count` | int  | Number of delete files which were removed by this command                  |
+| `added_delete_files_count`     | int  | Number of delete files which were added by this command                    |
+| `rewritten_bytes_count`        | long | Count of bytes across delete files which were removed by this command      |
+| `added_bytes_count`            | long | Count of bytes across all new delete files which were added by this command |
+
+
+#### Examples
+
+Rewrite position delete files in table `db.sample`.  This selects position delete files that fit default rewrite criteria, and writes new files of target size `target-file-size-bytes`.  Dangling deletes are removed from rewritten delete files.
+```sql
+CALL catalog_name.system.rewrite_position_delete_files('db.sample')
+```
+
+Rewrite all position delete files in table `db.sample`, writing new files `target-file-size-bytes`.   Dangling deletes are removed from rewritten delete files.
+```sql
+CALL catalog_name.system.rewrite_position_delete_files(table => 'db.sample', options => map('rewrite-all', 'true'))
+```
+
+Rewrite position delete files in table `db.sample`.  This selects position delete files in partitions where 2 or more position delete files need to be rewritten based on size criteria.  Dangling deletes are removed from rewritten delete files.
+```sql
+CALL catalog_name.system.rewrite_position_delete_files(table => 'db.sample', options => map('min-input-files','2'))
+```
+
 ## Table migration
 
 The `snapshot` and `migrate` procedures help test and migrate existing Hive or Spark tables to Iceberg.
@@ -437,6 +558,7 @@
 | `table`       | ✔️  | string | Name of the table to migrate |
 | `properties`  | ️   | map<string, string> | Properties for the new Iceberg table |
 | `drop_backup` |   | boolean | When true, the original table will not be retained as backup (defaults to false) |
+| `backup_table_name` |  | string | Name of the table that will be retained as backup (defaults to `table_BACKUP_`) |
 
 #### Output
 
@@ -467,6 +589,10 @@
 `expire_snapshot` calls will be able to physically delete the added files. This method should not be used if 
 `migrate` or `snapshot` are possible.
 
+{{< hint warning >}}
+Keep in mind the `add_files` procedure will fetch the Parquet metadata from each file being added just once. If you're using tiered storage, (such as [Amazon S3 Intelligent-Tiering storage class](https://aws.amazon.com/s3/storage-classes/intelligent-tiering/)), the underlying, file will be retrieved from the archive, and will remain on a higher tier for a set period of time.
+{{< /hint >}}
+
 #### Usage
 
 | Argument Name           | Required? | Type                | Description                                                                                         |
@@ -596,14 +722,15 @@
 
 #### Usage
 
-| Argument Name | Required? | Type | Description                                                                                                                                                                                                           |
-|---------------|----------|------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| `table`       | ✔️ | string | Name of the source table for the changelog                                                                                                                                                                            |
-| `changelog_view`        |   | string | Name of the view to create                                                                                                                                                                                            |
-| `options`     |   | map<string, string> | A map of Spark read options to use                                                                                                                                                                                    |
-|`compute_updates`| | boolean | Whether to compute pre/post update images (see below for more information). Defaults to false.                                                                                                                        | 
-|`identifier_columns`| | array<string> | The list of identifier columns to compute updates. If the argument `compute_updates` is set to true and `identifier_columns` are not provided, the table’s current identifier fields will be used to compute updates. |
-|`remove_carryovers`| | boolean | Whether to remove carry-over rows (see below for more information). Defaults to true.                                                                                                                                 |
+| Argument Name        | Required? | Type                | Description                                                                                                                                                                                          |
+|----------------------|-----------|---------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| `table`              | ✔️         | string              | Name of the source table for the changelog                                                                                                                                                           |
+| `changelog_view`     |           | string              | Name of the view to create                                                                                                                                                                           |
+| `options`            |           | map<string, string> | A map of Spark read options to use                                                                                                                                                                   |
+| `net_changes`        |           | boolean             | Whether to output net changes (see below for more information). Defaults to false.                                                                                                                   |
+| `compute_updates`    |           | boolean             | Whether to compute pre/post update images (see below for more information). Defaults to false.                                                                                                       | 
+| `identifier_columns` |           | array<string>       | The list of identifier columns to compute updates. If the argument `compute_updates` is set to true and `identifier_columns` are not provided, the table’s current identifier fields will be used.   |
+| `remove_carryovers`  |           | boolean             | Whether to remove carry-over rows (see below for more information). Defaults to true. Deprecated since 1.4.0, will be removed in 1.5.0;  Please query `SparkChangelogTable` to view carry-over rows. |
 
 Here is a list of commonly used Spark read options:
 * `start-snapshot-id`: the exclusive start snapshot ID. If not provided, it reads from the table’s first snapshot inclusively. 
@@ -666,6 +793,22 @@
 |2	| Bob	   |INSERT	|0	|5390529835796506035|
 |1	| Alice  |DELETE	|1	|8764748981452218370|
 
+Create a changelog view that computes net changes. It removes intermediate changes and only outputs the net changes. 
+```sql
+CALL spark_catalog.system.create_changelog_view(
+  table => 'db.tbl',
+  options => map('end-snapshot-id', '87647489814522183702'),
+  net_changes => true
+)
+```
+
+With the net changes, the above changelog view only contains the following row since Alice was inserted in the first snapshot and deleted in the second snapshot.
+
+|  id	| name	  |_change_type |	_change_ordinal	| _change_snapshot_id |
+|---|--------|---|---|---|
+|2	| Bob	   |INSERT	|0	|5390529835796506035|
+
+
 #### Carry-over Rows
 
 The procedure removes the carry-over rows by default. Carry-over rows are the result of row-level operations(`MERGE`, `UPDATE` and `DELETE`)
@@ -678,8 +821,10 @@
 | 1   | Alice | DELETE       |
 | 1   | Alice | INSERT       |
 
-By default, this view finds the carry-over rows and removes them from the result. User can disable this
-behavior by setting the `remove_carryovers` option to `false`.
+To see carry-over rows, query `SparkChangelogTable` as follows:
+```sql
+SELECT * FROM spark_catalog.db.tbl.changes
+```
 
 #### Pre/Post Update Images
 
@@ -702,4 +847,4 @@
 | id  | name   | _change_type |
 |-----|--------|--------------|
 | 3   | Robert | UPDATE_BEFORE|
-| 3   | Dan    | UPDATE_AFTER |
\ No newline at end of file
+| 3   | Dan    | UPDATE_AFTER |
diff --git a/docs/content/spark-queries.md b/docs/content/spark-queries.md
index f2ebf1d..2923638 100644
--- a/docs/content/spark-queries.md
+++ b/docs/content/spark-queries.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_queries
         weight: 0
 ---
 <!--
@@ -269,17 +270,29 @@
 
 ### Files
 
-To show a table's current data files:
+To show a table's current files:
 
 ```sql
 SELECT * FROM prod.db.table.files;
 ```
 
-|content|file_path                                                                                                                                   |file_format|spec_id|partition|record_count|file_size_in_bytes|column_sizes      |value_counts    |null_value_counts|nan_value_counts|lower_bounds           |upper_bounds           |key_metadata|split_offsets|equality_ids|sort_order_id|
+| content | file_path | file_format | spec_id | record_count | file_size_in_bytes | column_sizes | value_counts | null_value_counts | nan_value_counts | lower_bounds | upper_bounds | key_metadata | split_offsets | equality_ids | sort_order_id | readable_metrics |
 | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- | -- |
-| 0 | s3:/.../table/data/00000-3-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET   | 0  | {1999-01-01, 01} | 1            | 597                | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0]  | []               | [1 -> , 2 -> c] | [1 -> , 2 -> c] | null         | [4]           | null | null |
-| 0 | s3:/.../table/data/00001-4-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET   | 0  | {1999-01-01, 02} | 1            | 597                | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0]  | []               | [1 -> , 2 -> b] | [1 -> , 2 -> b] | null         | [4]           | null | null |
-| 0 | s3:/.../table/data/00002-5-8d6d60e8-d427-4809-bcf0-f5d45a4aad96.parquet | PARQUET   | 0  | {1999-01-01, 03} | 1            | 597                | [1 -> 90, 2 -> 62] | [1 -> 1, 2 -> 1] | [1 -> 0, 2 -> 0]  | []               | [1 -> , 2 -> a] | [1 -> , 2 -> a] | null         | [4]           | null | null |
+| 0 | s3:/.../table/data/00042-3-a9aa8b24-20bc-4d56-93b0-6b7675782bb5-00001.parquet | PARQUET | 0 | 1 | 652 | {1:52,2:48} | {1:1,2:1} | {1:0,2:0} | {} | {1:,2:d} | {1:,2:d} | NULL | [4] | NULL | 0 | {"data":{"column_size":48,"value_count":1,"null_value_count":0,"nan_value_count":null,"lower_bound":"d","upper_bound":"d"},"id":{"column_size":52,"value_count":1,"null_value_count":0,"nan_value_count":null,"lower_bound":1,"upper_bound":1}} |
+| 0 | s3:/.../table/data/00000-0-f9709213-22ca-4196-8733-5cb15d2afeb9-00001.parquet | PARQUET | 0 | 1 | 643 | {1:46,2:48} | {1:1,2:1} | {1:0,2:0} | {} | {1:,2:a} | {1:,2:a} | NULL | [4] | NULL | 0 | {"data":{"column_size":48,"value_count":1,"null_value_count":0,"nan_value_count":null,"lower_bound":"a","upper_bound":"a"},"id":{"column_size":46,"value_count":1,"null_value_count":0,"nan_value_count":null,"lower_bound":1,"upper_bound":1}} | 
+| 0 | s3:/.../table/data/00001-1-f9709213-22ca-4196-8733-5cb15d2afeb9-00001.parquet | PARQUET | 0 | 2 | 644 | {1:49,2:51} | {1:2,2:2} | {1:0,2:0} | {} | {1:,2:b} | {1:,2:c} | NULL | [4] | NULL | 0 | {"data":{"column_size":51,"value_count":2,"null_value_count":0,"nan_value_count":null,"lower_bound":"b","upper_bound":"c"},"id":{"column_size":49,"value_count":2,"null_value_count":0,"nan_value_count":null,"lower_bound":2,"upper_bound":3}} |
+| 1 | s3:/.../table/data/00081-4-a9aa8b24-20bc-4d56-93b0-6b7675782bb5-00001-deletes.parquet | PARQUET | 0 | 1 | 1560 | {2147483545:46,2147483546:152} | {2147483545:1,2147483546:1} | {2147483545:0,2147483546:0} | {} | {2147483545:,2147483546:s3:/.../table/data/00000-0-f9709213-22ca-4196-8733-5cb15d2afeb9-00001.parquet} | {2147483545:,2147483546:s3:/.../table/data/00000-0-f9709213-22ca-4196-8733-5cb15d2afeb9-00001.parquet} | NULL | [4] | NULL | NULL | {"data":{"column_size":null,"value_count":null,"null_value_count":null,"nan_value_count":null,"lower_bound":null,"upper_bound":null},"id":{"column_size":null,"value_count":null,"null_value_count":null,"nan_value_count":null,"lower_bound":null,"upper_bound":null}} |
+| 2 | s3:/.../table/data/00047-25-833044d0-127b-415c-b874-038a4f978c29-00612.parquet | PARQUET | 0 | 126506 | 28613985 | {100:135377,101:11314} | {100:126506,101:126506} | {100:105434,101:11} | {} | {100:0,101:17} | {100:404455227527,101:23} | NULL | NULL | [1] | 0 | {"id":{"column_size":135377,"value_count":126506,"null_value_count":105434,"nan_value_count":null,"lower_bound":0,"upper_bound":404455227527},"data":{"column_size":11314,"value_count":126506,"null_value_count": 11,"nan_value_count":null,"lower_bound":17,"upper_bound":23}} |
+
+{{< hint info >}}
+Content refers to type of content stored by the data file:
+  0  Data
+  1  Position Deletes
+  2  Equality Deletes
+{{< /hint >}}
+
+To show only data files or delete files, query `prod.db.table.data_files` and `prod.db.table.delete_files` respectively.
+To show all files, data files and delete files across all tracked snapshots, query `prod.db.table.all_files`, `prod.db.table.all_data_files` and `prod.db.table.all_delete_files` respectively.
 
 ### Manifests
 
@@ -310,15 +323,15 @@
 SELECT * FROM prod.db.table.partitions;
 ```
 
-| partition | record_count | file_count | spec_id |
-| -- | -- | -- | -- |
-|  {20211001, 11}|           1|         1|         0|
-|  {20211002, 11}|           1|         1|         0|
-|  {20211001, 10}|           1|         1|         0|
-|  {20211002, 10}|           1|         1|         0|
+| partition      | spec_id | record_count  | file_count | total_data_file_size_in_bytes | position_delete_record_count | position_delete_file_count | equality_delete_record_count | equality_delete_file_count | last_updated_at(μs) | last_updated_snapshot_id |
+| -------------- |---------|---------------|------------|--------------------------|------------------------------|----------------------------|------------------------------|----------------------------|---------------------|--------------------------|
+| {20211001, 11} | 0       | 1             | 1          | 100                      | 2                            | 1                          | 0                            | 0                          | 1633086034192000    | 9205185327307503337      |
+| {20211002, 11} | 0       | 4             | 3          | 500                      | 1                            | 1                          | 0                            | 0                          | 1633172537358000    | 867027598972211003       |
+| {20211001, 10} | 0       | 7             | 4          | 700                      | 0                            | 0                          | 0                            | 0                          | 1633082598716000    | 3280122546965981531      |
+| {20211002, 10} | 0       | 3             | 2          | 400                      | 0                            | 0                          | 1                            | 1                          | 1633169159489000    | 6941468797545315876      |
 
 Note:
-1. For unpartitioned tables, the partitions table will contain only the record_count and file_count columns.
+1. For unpartitioned tables, the partitions table will not contain the partition and spec_id fields.
 
 2. The partitions metadata table shows partitions with data files or delete files in the current snapshot. However, delete files are not applied, and so in some cases partitions may be shown even though all their data rows are marked deleted by delete files.
 
diff --git a/docs/content/spark-structured-streaming.md b/docs/content/spark-structured-streaming.md
index 77a7960..b4aed33 100644
--- a/docs/content/spark-structured-streaming.md
+++ b/docs/content/spark-structured-streaming.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_structured_streaming
         weight: 0
 ---
 <!--
@@ -27,10 +28,7 @@
 
 # Spark Structured Streaming
 
-Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API
-with different levels of support in Spark versions.
-
-As of Spark 3, DataFrame reads and writes are supported.
+Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API with different levels of support in Spark versions.
 
 ## Streaming Reads
 
@@ -52,77 +50,79 @@
 To write values from streaming query to Iceberg table, use `DataStreamWriter`:
 
 ```scala
-val tableIdentifier: String = ...
 data.writeStream
     .format("iceberg")
     .outputMode("append")
     .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES))
-    .option("path", tableIdentifier)
+    .option("checkpointLocation", checkpointPath)
+    .toTable("database.table_name")
+```
+
+If you're using Spark 3.0 or earlier, you need to use `.option("path", "database.table_name").start()`, instead of `.toTable("database.table_name")`.
+
+In the case of the directory-based Hadoop catalog:
+
+```scala
+data.writeStream
+    .format("iceberg")
+    .outputMode("append")
+    .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES))
+    .option("path", "hdfs://nn:8020/path/to/table") 
     .option("checkpointLocation", checkpointPath)
     .start()
 ```
 
-The `tableIdentifier` can be:
-
-* The fully-qualified path to a HDFS table, like `hdfs://nn:8020/path/to/table`
-* A table name if the table is tracked by a catalog, like `database.table_name`
-
-Iceberg doesn't support "continuous processing", as it doesn't provide the interface to "commit" the output.
-
 Iceberg supports `append` and `complete` output modes:
 
 * `append`: appends the rows of every micro-batch to the table
 * `complete`: replaces the table contents every micro-batch
 
-The table should be created in prior to start the streaming query. Refer [SQL create table](../spark-ddl/#create-table)
-on Spark page to see how to create the Iceberg table.
+Prior to starting the streaming query, ensure you created the table. Refer to the [SQL create table](../spark-ddl/#create-table) documentation to learn how to create the Iceberg table.
 
-### Writing against partitioned table
+Iceberg doesn't support experimental [continuous processing](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#continuous-processing), as it doesn't provide the interface to "commit" the output.
 
-Iceberg requires the data to be sorted according to the partition spec per task (Spark partition) in prior to write
+### Partitioned table
+
+Iceberg requires sorting data by partition per task prior to writing the data. In Spark tasks are split by Spark partition.
 against partitioned table. For batch queries you're encouraged to do explicit sort to fulfill the requirement
 (see [here](../spark-writes/#writing-to-partitioned-tables)), but the approach would bring additional latency as
 repartition and sort are considered as heavy operations for streaming workload. To avoid additional latency, you can
 enable fanout writer to eliminate the requirement.
 
 ```scala
-val tableIdentifier: String = ...
 data.writeStream
     .format("iceberg")
     .outputMode("append")
     .trigger(Trigger.ProcessingTime(1, TimeUnit.MINUTES))
-    .option("path", tableIdentifier)
     .option("fanout-enabled", "true")
     .option("checkpointLocation", checkpointPath)
-    .start()
+    .toTable("database.table_name")
 ```
 
-Fanout writer opens the files per partition value and doesn't close these files till write task is finished.
-This functionality is discouraged for batch query, as explicit sort against output rows isn't expensive for batch workload.
+Fanout writer opens the files per partition value and doesn't close these files till the write task finishes. Avoid using the fanout writer for batch writing, as explicit sort against output rows is cheap for batch workloads.
 
 ## Maintenance for streaming tables
 
-Streaming queries can create new table versions quickly, which creates lots of table metadata to track those versions.
+Streaming writes can create new table versions quickly, creating lots of table metadata to track those versions.
 Maintaining metadata by tuning the rate of commits, expiring old snapshots, and automatically cleaning up metadata files
 is highly recommended.
 
 ### Tune the rate of commits
 
-Having high rate of commits would produce lots of data files, manifests, and snapshots which leads the table hard
-to maintain. We encourage having trigger interval 1 minute at minimum, and increase the interval if needed.
+Having a high rate of commits produces data files, manifests, and snapshots which leads to additional maintenance. It is recommended to have a trigger interval of 1 minute at the minimum and increase the interval if needed.
 
 The triggers section in [Structured Streaming Programming Guide](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#triggers)
 documents how to configure the interval.
 
 ### Expire old snapshots
 
-Each micro-batch written to a table produces a new snapshot, which are tracked in table metadata until they are expired to remove the metadata and any data files that are no longer needed. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](../maintenance#expire-snapshots).
+Each batch written to a table produces a new snapshot. Iceberg tracks snapshots in table metadata until they are expired. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](../maintenance#expire-snapshots). [Snapshot expiration](../spark-procedures/#expire_snapshots) is the procedure of removing the metadata and any data files that are no longer needed. By default, the procedure will expire the snapshots older than five days. 
 
 ### Compacting data files
 
-The amount of data written in a micro batch is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](../maintenance#compact-data-files) reduces the metadata needed by the table, and increases query efficiency.
+The amount of data written from a streaming process is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](../maintenance#compact-data-files) reduces the metadata needed by the table, and increases query efficiency. Iceberg and Spark [comes with the `rewrite_data_files` procedure](../spark-procedures/#rewrite_data_files).
 
 ### Rewrite manifests
 
-To optimize write latency on streaming workload, Iceberg may write the new snapshot with a "fast" append that does not automatically compact manifests.
-This could lead lots of small manifest files. Manifests can be [rewritten to optimize queries and to compact](../maintenance#rewrite-manifests).
+To optimize write latency on a streaming workload, Iceberg can write the new snapshot with a "fast" append that does not automatically compact manifests.
+This could lead lots of small manifest files. Iceberg can [rewrite the number of manifest files to improve query performance](../maintenance#rewrite-manifests). Iceberg and Spark [come with the `rewrite_manifests` procedure](../spark-procedures/#rewrite_manifests).
diff --git a/docs/content/spark-writes.md b/docs/content/spark-writes.md
index 87a5845..7338af0 100644
--- a/docs/content/spark-writes.md
+++ b/docs/content/spark-writes.md
@@ -6,6 +6,7 @@
 menu:
     main:
         parent: Spark
+        identifier: spark_writes
         weight: 0
 ---
 <!--
@@ -33,16 +34,16 @@
 
 Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API with different levels of support in Spark versions:
 
-| Feature support                                  | Spark 3 | Notes                                        |
-|--------------------------------------------------|-----------|----------------------------------------------|
-| [SQL insert into](#insert-into)                  | ✔️        |                                              |
-| [SQL merge into](#merge-into)                    | ✔️        | ⚠ Requires Iceberg Spark extensions          |
-| [SQL insert overwrite](#insert-overwrite)        | ✔️        |                                              |
-| [SQL delete from](#delete-from)                  | ✔️        | ⚠ Row-level delete requires Spark extensions |
-| [SQL update](#update)                            | ✔️        | ⚠ Requires Iceberg Spark extensions          |
-| [DataFrame append](#appending-data)              | ✔️        |                                              |
-| [DataFrame overwrite](#overwriting-data)         | ✔️        |                                              |
-| [DataFrame CTAS and RTAS](#creating-tables)      | ✔️        |                                              |
+| Feature support                                  | Spark 3 | Notes                                                                       |
+|--------------------------------------------------|-----------|-----------------------------------------------------------------------------|
+| [SQL insert into](#insert-into)                  | ✔️        | ⚠ Requires `spark.sql.storeAssignmentPolicy=ANSI` (default since Spark 3.0) |
+| [SQL merge into](#merge-into)                    | ✔️        | ⚠ Requires Iceberg Spark extensions                                         |
+| [SQL insert overwrite](#insert-overwrite)        | ✔️        | ⚠ Requires `spark.sql.storeAssignmentPolicy=ANSI` (default since Spark 3.0) |
+| [SQL delete from](#delete-from)                  | ✔️        | ⚠ Row-level delete requires Iceberg Spark extensions                        |
+| [SQL update](#update)                            | ✔️        | ⚠ Requires Iceberg Spark extensions                                         |
+| [DataFrame append](#appending-data)              | ✔️        |                                                                             |
+| [DataFrame overwrite](#overwriting-data)         | ✔️        |                                                                             |
+| [DataFrame CTAS and RTAS](#creating-tables)      | ✔️        | ⚠ Requires DSv2 API                                                         |
 
 
 ## Writing with SQL
@@ -183,8 +184,6 @@
 
 ### `UPDATE`
 
-Spark 3.1 added support for `UPDATE` queries that update matching rows in tables.
-
 Update queries accept a filter to match rows to update.
 
 ```sql
@@ -209,7 +208,7 @@
 Note WAP branch and branch identifier cannot both be specified.
 Also, the branch must exist before performing the write. 
 The operation does **not** create the branch if it does not exist. 
-For more information on branches please refer to [branches](../../tables/branching)
+For more information on branches please refer to [branches](../tables/branching)
  
 ```sql
 -- INSERT (1,' a') (2, 'b') into the audit branch.
@@ -312,20 +311,39 @@
     .createOrReplace()
 ```
 
-## Writing to partitioned tables
+### Schema Merge
 
-Iceberg requires the data to be sorted according to the partition spec per task (Spark partition) in prior to write
-against partitioned table. This applies both Writing with SQL and Writing with DataFrames.
+While inserting or updating Iceberg is capable of resolving schema mismatch at runtime. If configured, Iceberg will perform an automatic schema evolution as follows:
 
-{{< hint info >}}
-Explicit sort is necessary because Spark doesn't allow Iceberg to request a sort before writing as of Spark 3.0.
-[SPARK-23889](https://issues.apache.org/jira/browse/SPARK-23889) is filed to enable Iceberg to require specific
-distribution & sort order to Spark.
-{{< /hint >}}
 
-{{< hint info >}}
-Both global sort (`orderBy`/`sort`) and local sort (`sortWithinPartitions`) work for the requirement.
-{{< /hint >}}
+* A new column is present in the source but not in the target table.
+    
+  The new column is added to the target table. Column values are set to `NULL` in all the rows already present in the table
+
+* A column is present in the target but not in the source. 
+
+  The target column value is set to `NULL` when inserting or left unchanged when updating the row.
+
+The target table must be configured to accept any schema change by setting the property `write.spark.accept-any-schema` to `true`.
+
+```sql
+ALTER TABLE prod.db.sample SET TBLPROPERTIES (
+  'write.spark.accept-any-schema'='true'
+)
+```
+The writer must enable the `mergeSchema` option.
+
+```scala
+data.writeTo("prod.db.sample").option("mergeSchema","true").append()
+```
+
+
+## Writing Distribution Modes
+
+Iceberg's default Spark writers require that the data in each spark task is clustered by partition values. This 
+distribution is required to minimize the number of file handles that are held open while writing. By default, starting
+in Iceberg 1.2.0, Iceberg also requests that Spark pre-sort data to be written to fit this distribution. The
+request to Spark is done through the table property `write.distribution-mode` with the value `hash`.
 
 Let's go through writing the data against below sample table:
 
@@ -339,74 +357,61 @@
 PARTITIONED BY (days(ts), category)
 ```
 
-To write data to the sample table, your data needs to be sorted by `days(ts), category`.
-
-If you're inserting data with SQL statement, you can use `ORDER BY` to achieve it, like below:
+To write data to the sample table, data needs to be sorted by `days(ts), category` but this is taken care
+of automatically by the default `hash` distribution. Previously this would have required manually sorting, but this 
+is no longer the case.
 
 ```sql
 INSERT INTO prod.db.sample
 SELECT id, data, category, ts FROM another_table
-ORDER BY ts, category
 ```
 
-If you're inserting data with DataFrame, you can use either `orderBy`/`sort` to trigger global sort, or `sortWithinPartitions`
-to trigger local sort. Local sort for example:
 
-```scala
-data.sortWithinPartitions("ts", "category")
-    .writeTo("prod.db.sample")
-    .append()
-```
+There are 3 options for `write.distribution-mode`
 
-You can simply add the original column to the sort condition for the most partition transformations, except `bucket`.
+* `none` - This is the previous default for Iceberg.  
+This mode does not request any shuffles or sort to be performed automatically by Spark. Because no work is done 
+automatically by Spark, the data must be *manually* sorted by partition value. The data must be sorted either within 
+each spark task, or globally within the entire dataset. A global sort will minimize the number of output files.  
+A sort can be avoided by using the Spark [write fanout](#write-properties) property but this will cause all 
+file handles to remain open until each write task has completed.
+* `hash` - This mode is the new default and requests that Spark uses a hash-based exchange to shuffle the incoming
+write data before writing.  
+Practically, this means that each row is hashed based on the row's partition value and then placed
+in a corresponding Spark task based upon that value. Further division and coalescing of tasks may take place because of
+[Spark's Adaptive Query planning](#controlling-file-sizes).
+* `range` - This mode requests that Spark perform a range based exchanged to shuffle the data before writing.  
+This is a two stage procedure which is more expensive than the `hash` mode. The first stage samples the data to 
+be written based on the partition and sort columns. The second stage uses the range information to shuffle the input data into Spark 
+tasks. Each task gets an exclusive range of the input data which clusters the data by partition and also globally sorts.  
+While this is more expensive than the hash distribution, the global ordering can be beneficial for read performance if
+sorted columns are used during queries. This mode is used by default if a table is created with a 
+sort-order. Further division and coalescing of tasks may take place because of
+[Spark's Adaptive Query planning](#controlling-file-sizes).
 
-For `bucket` partition transformation, you need to register the Iceberg transform function in Spark to specify it during sort.
 
-Let's go through another sample table having bucket partition:
+## Controlling File Sizes
 
-```sql
-CREATE TABLE prod.db.sample (
-    id bigint,
-    data string,
-    category string,
-    ts timestamp)
-USING iceberg
-PARTITIONED BY (bucket(16, id))
-```
+When writing data to Iceberg with Spark, it's important to note that Spark cannot write a file larger than a Spark 
+task and a file cannot span an Iceberg partition boundary. This means although Iceberg will always roll over a file 
+when it grows to [`write.target-file-size-bytes`](../configuration/#write-properties), but unless the Spark task is 
+large enough that will not happen. The size of the file created on disk will also be much smaller than the Spark task 
+since the on disk data will be both compressed and in columnar format as opposed to Spark's uncompressed row 
+representation. This means a 100 megabyte Spark task will create a file much smaller than 100 megabytes even if that
+task is writing to a single Iceberg partition. If the task writes to multiple partitions, the files will be even
+smaller than that.
 
-You need to register the function to deal with bucket, like below:
+To control what data ends up in each Spark task use a [`write distribution mode`](#writing-distribution-modes) 
+or manually repartition the data. 
 
-```scala
-import org.apache.iceberg.spark.IcebergSpark
-import org.apache.spark.sql.types.DataTypes
-
-IcebergSpark.registerBucketUDF(spark, "iceberg_bucket16", DataTypes.LongType, 16)
-```
-
-{{< hint info >}}
-Explicit registration of the function is necessary because Spark doesn't allow Iceberg to provide functions.
-[SPARK-27658](https://issues.apache.org/jira/browse/SPARK-27658) is filed to enable Iceberg to provide functions
-which can be used in query.
-{{< /hint >}}
-
-Here we just registered the bucket function as `iceberg_bucket16`, which can be used in sort clause.
-
-If you're inserting data with SQL statement, you can use the function like below:
-
-```sql
-INSERT INTO prod.db.sample
-SELECT id, data, category, ts FROM another_table
-ORDER BY iceberg_bucket16(id)
-```
-
-If you're inserting data with DataFrame, you can use the function like below:
-
-```scala
-data.sortWithinPartitions(expr("iceberg_bucket16(id)"))
-    .writeTo("prod.db.sample")
-    .append()
-```
-
+To adjust Spark's task size it is important to become familiar with Spark's various Adaptive Query Execution (AQE) 
+parameters. When the `write.distribution-mode` is not `none`, AQE will control the coalescing and splitting of Spark
+tasks during the exchange to try to create tasks of `spark.sql.adaptive.advisoryPartitionSizeInBytes` size. These 
+settings will also affect any user performed re-partitions or sorts. 
+It is important again to note that this is the in-memory Spark row size and not the on disk
+columnar-compressed size, so a larger value than the target file size will need to be specified. The ratio of 
+in-memory size to on disk size is data dependent. Future work in Spark should allow Iceberg to automatically adjust this
+parameter at write time to match the `write.target-file-size-bytes`.
 
 ## Type compatibility
 
@@ -417,25 +422,26 @@
 
 This type conversion table describes how Spark types are converted to the Iceberg types. The conversion applies on both creating Iceberg table and writing to Iceberg table via Spark.
 
-| Spark           | Iceberg                 | Notes |
-|-----------------|-------------------------|-------|
-| boolean         | boolean                 |       |
-| short           | integer                 |       |
-| byte            | integer                 |       |
-| integer         | integer                 |       |
-| long            | long                    |       |
-| float           | float                   |       |
-| double          | double                  |       |
-| date            | date                    |       |
-| timestamp       | timestamp with timezone |       |
-| char            | string                  |       |
-| varchar         | string                  |       |
-| string          | string                  |       |
-| binary          | binary                  |       |
-| decimal         | decimal                 |       |
-| struct          | struct                  |       |
-| array           | list                    |       |
-| map             | map                     |       |
+| Spark           | Iceberg                    | Notes |
+|-----------------|----------------------------|-------|
+| boolean         | boolean                    |       |
+| short           | integer                    |       |
+| byte            | integer                    |       |
+| integer         | integer                    |       |
+| long            | long                       |       |
+| float           | float                      |       |
+| double          | double                     |       |
+| date            | date                       |       |
+| timestamp       | timestamp with timezone    |       |
+| timestamp_ntz    | timestamp without timezone |       |
+| char            | string                     |       |
+| varchar         | string                     |       |
+| string          | string                     |       |
+| binary          | binary                     |       |
+| decimal         | decimal                    |       |
+| struct          | struct                     |       |
+| array           | list                       |       |
+| map             | map                        |       |
 
 {{< hint info >}}
 The table is based on representing conversion during creating table. In fact, broader supports are applied on write. Here're some points on write:
@@ -458,7 +464,7 @@
 | date                       | date                    |               |
 | time                       |                         | Not supported |
 | timestamp with timezone    | timestamp               |               |
-| timestamp without timezone |                         | Not supported |
+| timestamp without timezone | timestamp_ntz            |               |
 | string                     | string                  |               |
 | uuid                       | string                  |               |
 | fixed                      | binary                  |               |
diff --git a/docs/content/table-migration.md b/docs/content/table-migration.md
index 9635a64..0eacd51 100644
--- a/docs/content/table-migration.md
+++ b/docs/content/table-migration.md
@@ -4,6 +4,7 @@
 menu:
   main:
     parent: "Migration"
+    identifier: table_migration
     weight: 100
 ---
 <!--