This is an automated email from the ASF dual-hosted git repository.

szehon pushed a commit to branch 1.3.1
in repository https://gitbox.apache.org/repos/asf/iceberg-docs.git


The following commit(s) were added to refs/heads/1.3.1 by this push:
     new 67101799 Sync with 1.3.1 docs
67101799 is described below

commit 67101799c6a1577ab99b41a285c5f29e2b66dfdb
Author: Szehon Ho <[email protected]>
AuthorDate: Mon Jul 24 17:18:21 2023 -0700

    Sync with 1.3.1 docs
---
 docs/content/aws.md                        |  72 ++++++++--------
 docs/content/branching-and-tagging.md      |   3 +-
 docs/content/configuration.md              | 104 +++++++++++++----------
 docs/content/dell.md                       |   1 +
 docs/content/delta-lake-migration.md       |   1 +
 docs/content/evolution.md                  |   1 +
 docs/content/flink-actions.md              |   1 +
 docs/content/flink-configuration.md        |  46 ++++++-----
 docs/content/flink-connector.md            |   1 +
 docs/content/flink-ddl.md                  |   1 +
 docs/content/flink-getting-started.md      |   1 +
 docs/content/flink-queries.md              |   1 +
 docs/content/flink-writes.md               |   1 +
 docs/content/hive-migration.md             |   1 +
 docs/content/java-api-quickstart.md        |   1 +
 docs/content/java-api.md                   |   1 +
 docs/content/java-custom-catalog.md        |   1 +
 docs/content/jdbc.md                       |   1 +
 docs/content/maintenance.md                |   1 +
 docs/content/nessie.md                     |   1 +
 docs/content/partitioning.md               |   1 +
 docs/content/performance.md                |   1 +
 docs/content/reliability.md                |   1 +
 docs/content/schemas.md                    |   1 +
 docs/content/spark-configuration.md        |  19 ++++-
 docs/content/spark-ddl.md                  |   1 +
 docs/content/spark-getting-started.md      |   1 +
 docs/content/spark-procedures.md           |  46 +++++++++++
 docs/content/spark-queries.md              |   1 +
 docs/content/spark-structured-streaming.md |   1 +
 docs/content/spark-writes.md               | 128 ++++++++++++-----------------
 docs/content/table-migration.md            |   1 +
 32 files changed, 262 insertions(+), 181 deletions(-)

diff --git a/docs/content/aws.md b/docs/content/aws.md
index 2f8e1911..e1eb5699 100644
--- a/docs/content/aws.md
+++ b/docs/content/aws.md
@@ -4,6 +4,7 @@ url: aws
 menu:
     main:
         parent: Integrations
+        identifier: aws_integration
         weight: 0
 ---
 <!--
@@ -55,7 +56,7 @@ For example, to use AWS features with Spark 3.3 (with scala 
2.12) and AWS client
 ICEBERG_VERSION={{% icebergVersion %}}
 
DEPENDENCIES="org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:$ICEBERG_VERSION"
 
-# add AWS dependnecy
+# add AWS dependency
 AWS_SDK_VERSION=2.20.18
 AWS_MAVEN_GROUP=software.amazon.awssdk
 AWS_PACKAGES=(
@@ -87,7 +88,7 @@ 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
+# download AWS dependency
 AWS_SDK_VERSION=2.20.18
 AWS_MAVEN_URL=$MAVEN_URL/software/amazon/awssdk
 AWS_PACKAGES=(
@@ -172,12 +173,12 @@ There is a unique Glue metastore in each AWS account and 
each AWS region.
 By default, `GlueCatalog` chooses the Glue metastore to use based on the 
user's default AWS client credential and region setup.
 You can specify the Glue catalog ID through `glue.id` catalog property to 
point to a Glue catalog in a different AWS account.
 The Glue catalog ID is your numeric AWS account ID.
-If the Glue catalog is in a different region, you should configure you AWS 
client to point to the correct region, 
+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.
@@ -197,9 +198,8 @@ By default, Iceberg uses Glue's optimistic locking for 
concurrent updates to a t
 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 +239,7 @@ OPTIONS ('location'='s3://my-special-table-bucket')
 PARTITIONED BY (category);
 ```
 
-For engines like Spark that supports the `LOCATION` keyword, the above SQL 
statement is equivalent to:
+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 +279,7 @@ The DynamoDB table is designed with the following columns:
 
 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 +288,25 @@ This design has the following benefits:
 
 ### 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 +345,7 @@ Here are the configurations that users can tune related to 
this feature:
 
 * 
[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 [...]
-* 
[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 +364,13 @@ For more details, please read [S3 ACL 
Documentation](https://docs.aws.amazon.com
 ### 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 +411,8 @@ There is no redundant consistency wait and check which 
might negatively impact p
 ### 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 +521,7 @@ For more details on using S3 Acceleration, please refer to 
[Configuring fast, se
 ### 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,7 +555,7 @@ This client factory has the following configurable catalog 
properties:
 | 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.
@@ -575,8 +575,8 @@ spark-sql --packages 
org.apache.iceberg:iceberg-spark-runtime:{{% icebergVersion
 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. 
+This HTTP client optimizes for minimum dependencies and startup latency but 
supports 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 
the cost of extra dependency and additional startup latency. 
 
 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 +606,15 @@ Apache HTTP Client has the following configurable 
properties:
 
 | 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
@@ -677,7 +677,7 @@ 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 f2d8606e..575c3667 100644
--- a/docs/content/branching-and-tagging.md
+++ b/docs/content/branching-and-tagging.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_branching
         weight: 0
 ---
 
@@ -30,7 +31,7 @@ menu:
 
 ## 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.
 
diff --git a/docs/content/configuration.md b/docs/content/configuration.md
index 8efd8fc1..7fa2d94a 100644
--- a/docs/content/configuration.md
+++ b/docs/content/configuration.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_configuration
         weight: 0
 ---
 <!--
@@ -46,51 +47,51 @@ Iceberg tables support table properties to configure table 
behavior, like the de
 
 ### 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
 
@@ -178,8 +179,21 @@ The HMS table locking is a 2-step process:
 | 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 173f2d18..af484f93 100644
--- a/docs/content/dell.md
+++ b/docs/content/dell.md
@@ -4,6 +4,7 @@ url: dell
 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 2c1dddf5..b0e7b221 100644
--- a/docs/content/delta-lake-migration.md
+++ b/docs/content/delta-lake-migration.md
@@ -4,6 +4,7 @@ url: delta-lake-migration
 menu:
   main:
     parent: "Migration"
+    identifier: delta_lake_migration
     weight: 300
 ---
 <!--
diff --git a/docs/content/evolution.md b/docs/content/evolution.md
index 3f3f18ee..7baf463e 100644
--- a/docs/content/evolution.md
+++ b/docs/content/evolution.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_evolution
         weight: 0
 ---
 <!--
diff --git a/docs/content/flink-actions.md b/docs/content/flink-actions.md
index 1fc5bb85..32bc3b9d 100644
--- a/docs/content/flink-actions.md
+++ b/docs/content/flink-actions.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_actions
         weight: 500
 ---
 <!--
diff --git a/docs/content/flink-configuration.md 
b/docs/content/flink-configuration.md
index 89741159..37db16c3 100644
--- a/docs/content/flink-configuration.md
+++ b/docs/content/flink-configuration.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_configuration
         weight: 600
 ---
 <!--
@@ -110,27 +111,28 @@ env.getConfig()
 
 `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 shou [...]
-| 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  [...]
+| 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 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') 
*/
 | 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 dc1520a5..aaf27fff 100644
--- a/docs/content/flink-connector.md
+++ b/docs/content/flink-connector.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_connector
         weight: 200
 ---
 <!--
diff --git a/docs/content/flink-ddl.md b/docs/content/flink-ddl.md
index 67f9e21d..30ff6b2f 100644
--- a/docs/content/flink-ddl.md
+++ b/docs/content/flink-ddl.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_ddl
         weight: 200
 ---
 <!--
diff --git a/docs/content/flink-getting-started.md 
b/docs/content/flink-getting-started.md
index 078bc124..fb67ea3e 100644
--- a/docs/content/flink-getting-started.md
+++ b/docs/content/flink-getting-started.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_getting_started
         weight: 100
 ---
 <!--
diff --git a/docs/content/flink-queries.md b/docs/content/flink-queries.md
index 2d62d18e..24ff608a 100644
--- a/docs/content/flink-queries.md
+++ b/docs/content/flink-queries.md
@@ -6,6 +6,7 @@ aliases:
 menu:
    main:
       parent: Flink
+      identifier: flink_queries
       weight: 300
 ---
 <!--
diff --git a/docs/content/flink-writes.md b/docs/content/flink-writes.md
index f61416bf..99442db4 100644
--- a/docs/content/flink-writes.md
+++ b/docs/content/flink-writes.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Flink
+        identifier: flink_writes
         weight: 400
 ---
 <!--
diff --git a/docs/content/hive-migration.md b/docs/content/hive-migration.md
index c4ef2562..21d38d78 100644
--- a/docs/content/hive-migration.md
+++ b/docs/content/hive-migration.md
@@ -4,6 +4,7 @@ url: hive-migration
 menu:
   main:
     parent: "Migration"
+    identifier: hive_migration
     weight: 200
 ---
 <!--
diff --git a/docs/content/java-api-quickstart.md 
b/docs/content/java-api-quickstart.md
index 24b61b42..358ed2b4 100644
--- a/docs/content/java-api-quickstart.md
+++ b/docs/content/java-api-quickstart.md
@@ -6,6 +6,7 @@ aliases:
 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 15731e41..62b51e09 100644
--- a/docs/content/java-api.md
+++ b/docs/content/java-api.md
@@ -6,6 +6,7 @@ aliases:
 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 0132dd18..59238a2b 100644
--- a/docs/content/java-custom-catalog.md
+++ b/docs/content/java-custom-catalog.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: "API"
+        identifier: java_custom_catalog
         weight: 300
 ---
 <!--
diff --git a/docs/content/jdbc.md b/docs/content/jdbc.md
index c096c29e..f7f70043 100644
--- a/docs/content/jdbc.md
+++ b/docs/content/jdbc.md
@@ -4,6 +4,7 @@ url: jdbc
 menu:
     main:
         parent: Integrations
+        identifier: jdbc_integration
         weight: 0
 ---
 <!--
diff --git a/docs/content/maintenance.md b/docs/content/maintenance.md
index 07a3099f..3e5e2a08 100644
--- a/docs/content/maintenance.md
+++ b/docs/content/maintenance.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_maintenance
         weight: 0
 ---
 <!--
diff --git a/docs/content/nessie.md b/docs/content/nessie.md
index b64847f7..da3c7a84 100644
--- a/docs/content/nessie.md
+++ b/docs/content/nessie.md
@@ -4,6 +4,7 @@ url: nessie
 menu:
     main:
         parent: Integrations
+        identifier: nessie_integration
         weight: 0
 ---
 <!--
diff --git a/docs/content/partitioning.md b/docs/content/partitioning.md
index 349d8faf..799fc4fc 100644
--- a/docs/content/partitioning.md
+++ b/docs/content/partitioning.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_partitioning
         weight: 0
 ---
 <!--
diff --git a/docs/content/performance.md b/docs/content/performance.md
index 91939877..f5e87e3a 100644
--- a/docs/content/performance.md
+++ b/docs/content/performance.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_performance
         weight: 0
 ---
 <!--
diff --git a/docs/content/reliability.md b/docs/content/reliability.md
index 6a115164..9a476684 100644
--- a/docs/content/reliability.md
+++ b/docs/content/reliability.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_reliability
         weight: 0
 ---
 <!--
diff --git a/docs/content/schemas.md b/docs/content/schemas.md
index 502d3306..03805fad 100644
--- a/docs/content/schemas.md
+++ b/docs/content/schemas.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Tables
+        identifier: tables_schema
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-configuration.md 
b/docs/content/spark-configuration.md
index 926ec020..866dd765 100644
--- a/docs/content/spark-configuration.md
+++ b/docs/content/spark-configuration.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_configuration
         weight: 0
 ---
 <!--
@@ -40,6 +41,14 @@ spark.sql.catalog.hive_prod.uri = 
thrift://metastore-host:port
 # 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 @@ Both catalogs are configured using properties nested under 
the catalog name. Com
 | 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).
 
diff --git a/docs/content/spark-ddl.md b/docs/content/spark-ddl.md
index b8622856..6c80b5b2 100644
--- a/docs/content/spark-ddl.md
+++ b/docs/content/spark-ddl.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_ddl
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-getting-started.md 
b/docs/content/spark-getting-started.md
index a9bbe834..f72bb7e7 100644
--- a/docs/content/spark-getting-started.md
+++ b/docs/content/spark-getting-started.md
@@ -7,6 +7,7 @@ aliases:
 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 382606d6..b1275add 100644
--- a/docs/content/spark-procedures.md
+++ b/docs/content/spark-procedures.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_procedures
         weight: 0
 ---
 <!--
@@ -364,6 +365,51 @@ Rewrite the manifests in table `db.sample` and disable the 
use of Spark caching.
 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 
|
+
+See the [`SizeBasedFileRewriter` Javadoc](../../../javadoc/{{% icebergVersion 
%}}/org/apache/iceberg/actions/SizeBasedFileRewriter.html#field.summary),
+for list of all the supported options for this procedure.
+
+Dangling deletes are always filtered out during rewriting.
+
+#### 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.
diff --git a/docs/content/spark-queries.md b/docs/content/spark-queries.md
index f2ebf1d8..535e5d62 100644
--- a/docs/content/spark-queries.md
+++ b/docs/content/spark-queries.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_queries
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-structured-streaming.md 
b/docs/content/spark-structured-streaming.md
index 77a79608..fda14b23 100644
--- a/docs/content/spark-structured-streaming.md
+++ b/docs/content/spark-structured-streaming.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_structured_streaming
         weight: 0
 ---
 <!--
diff --git a/docs/content/spark-writes.md b/docs/content/spark-writes.md
index 87a58452..5a312da7 100644
--- a/docs/content/spark-writes.md
+++ b/docs/content/spark-writes.md
@@ -6,6 +6,7 @@ aliases:
 menu:
     main:
         parent: Spark
+        identifier: spark_writes
         weight: 0
 ---
 <!--
@@ -312,20 +313,12 @@ data.writeTo("prod.db.table")
     .createOrReplace()
 ```
 
-## Writing to partitioned tables
+## Writing Distribution Modes
 
-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.
-
-{{< 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 >}}
+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 +332,61 @@ USING iceberg
 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()
 ```
 
-You can simply add the original column to the sort condition for the most 
partition transformations, except `bucket`.
-
-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:
-
-```sql
-CREATE TABLE prod.db.sample (
-    id bigint,
-    data string,
-    category string,
-    ts timestamp)
-USING iceberg
-PARTITIONED BY (bucket(16, id))
-```
-
-You need to register the function to deal with bucket, like below:
-
-```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()
-```
 
+There are 3 options for `write.distribution-mode`
+
+* `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).
+
+
+## Controlling File Sizes
+
+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.
+
+To control what data ends up in each Spark task use a [`write distribution 
mode`](#writing-distribution-modes) 
+or manually repartition the data. 
+
+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
 
diff --git a/docs/content/table-migration.md b/docs/content/table-migration.md
index 9635a643..0eacd519 100644
--- a/docs/content/table-migration.md
+++ b/docs/content/table-migration.md
@@ -4,6 +4,7 @@ url: table-migration
 menu:
   main:
     parent: "Migration"
+    identifier: table_migration
     weight: 100
 ---
 <!--


Reply via email to