You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by ja...@apache.org on 2023/05/01 00:33:22 UTC

[iceberg-docs] branch 1.2.1 updated: Update 1.2.1 to include branching/tagging docs and table migration docs (#230)

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

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


The following commit(s) were added to refs/heads/1.2.1 by this push:
     new 864d2c65 Update 1.2.1 to include branching/tagging docs and table migration docs (#230)
864d2c65 is described below

commit 864d2c65c4d6ca00070da13648315e4c731d4993
Author: Amogh Jahagirdar <ja...@amazon.com>
AuthorDate: Sun Apr 30 17:33:16 2023 -0700

    Update 1.2.1 to include branching/tagging docs and table migration docs (#230)
---
 docs/content/branching-and-tagging.md | 122 ++++++++++++++++++++++++++++++++++
 docs/content/delta-lake-migration.md  | 122 ++++++++++++++++++++++++++++++++++
 docs/content/flink-configuration.md   |   6 +-
 docs/content/flink-getting-started.md |  10 +++
 docs/content/flink-queries.md         |  46 +++++++++++++
 docs/content/flink-writes.md          |  10 +++
 docs/content/hive-migration.md        |  60 +++++++++++++++++
 docs/content/spark-ddl.md             |  62 ++++++++++++++++-
 docs/content/spark-writes.md          |  30 +++++++++
 docs/content/table-migration.md       |  79 ++++++++++++++++++++++
 10 files changed, 545 insertions(+), 2 deletions(-)

diff --git a/docs/content/branching-and-tagging.md b/docs/content/branching-and-tagging.md
new file mode 100644
index 00000000..f2d8606e
--- /dev/null
+++ b/docs/content/branching-and-tagging.md
@@ -0,0 +1,122 @@
+---
+title: "Branching and Tagging"
+url: branching
+aliases:
+    - "tables/branching"
+menu:
+    main:
+        parent: Tables
+        weight: 0
+---
+
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# Branching and Tagging
+
+## Overview
+
+Iceberg table metadata maintains a log of snapshots which represent the changes applied to a table.
+Snapshots are fundamental in Iceberg as they are the basis for reader isolation and time travel queries.
+For controlling metadata size and storage costs, Iceberg provides snapshot lifecycle management procedures such as [`expire_snapshots`](../../spark/spark-procedures/#expire-snapshots) for removing unused snapshots and no longer neccessary data files based on table snapshot retention properties.
+
+**For more sophisticated snapshot lifecycle management, Iceberg supports branches and tags which are named references to snapshots with their own independent lifecycles. This lifecycle is controlled by branch and tag level retention policies.** 
+Branches are independent lineages of snapshots and point to the head of the lineage. 
+Branches and tags have a maximum reference age property which control when the reference to the snapshot itself should be expired.
+Branches have retention properties which define the minimum number of snapshots to retain on a branch as well as the maximum age of individual snapshots to retain on the branch. 
+These properties are used when the expireSnapshots procedure is run. 
+For details on the algorithm for expireSnapshots, refer to the [spec](../../../spec#snapshot-retention-policy).
+
+## Use Cases
+
+Branching and tagging can be used for handling GDPR requirements and retaining important historical snapshots for auditing.
+Branches can also be used as part of data engineering workflows, for enabling experimental branches for testing and validating new jobs.
+See below for some examples of how branching and tagging can facilitate these use cases.
+
+### Historical Tags
+
+Tags can be used for retaining important historical snapshots for auditing purposes.
+
+![Historical Tags](../img/historical-snapshot-tag.png)
+
+The above diagram demonstrates retaininig important historical snapshot with the following retention policy, defined 
+via Spark SQL.
+
+1. Retain 1 snapshot per week for 1 month. This can be achieved by tagging the weekly snapshot and setting the tag retention to be a month.
+snapshots will be kept, and the branch reference itself will be retained for 1 week. 
+```sql
+-- Create a tag for the first end of week snapshot. Retain the snapshot for a week
+ALTER TABLE prod.db.table CREATE TAG 'EOW-01' AS OF VERSION 7 RETAIN 7 DAYS
+```
+
+2. Retain 1 snapshot per month for 6 months. This can be achieved by tagging the monthly snapshot and setting the tag retention to be 6 months.
+```sql
+-- Create a tag for the first end of month snapshot. Retain the snapshot for 6 months
+ALTER TABLE prod.db.table CREATE TAG 'EOM-01' AS OF VERSION 30 RETAIN 180 DAYS
+```
+
+3. Retain 1 snapshot per year forever. This can be achieved by tagging the annual snapshot. The default retention for branches and tags is forever.
+```sql
+-- Create a tag for the end of the year and retain it forever.
+ALTER TABLE prod.db.table CREATE TAG 'EOY-2023' AS OF VERSION 365
+```
+
+4. Create a temporary "test-branch" which is retained for 7 days and the latest 2 snapshots on the branch are retained.
+```sql
+-- Create a branch "test-branch" which will be retained for 7 days along with the  latest 2 snapshots
+ALTER TABLE prod.db.table CREATE BRANCH test-branch RETAIN 7 DAYS WITH RETENTION 2 SNAPSHOTS
+```
+
+### Audit Branch
+
+![Audit Branch](../img/audit-branch.png)
+
+The above diagram shows an example of using an audit branch for validating a write workflow. 
+
+1. First ensure `write.wap.enabled` is set.
+```sql
+ALTER TABLE db.table SET TBLPROPERTIES (
+    'write.wap.enabled''true'
+)
+```
+2. Create `audit-branch` starting from snapshot 3, which will be written to and retained for 1 week.
+```sql
+ALTER TABLE db.table CREATE BRANCH `audit-branch` AS OF VERSION 3 RETAIN 7 DAYS
+```
+3. Writes are performed on a separate `audit-branch` independent from the main table history.
+```sql
+-- WAP Branch write
+SET spark.wap.branch = 'audit-branch'
+INSERT INTO prod.db.table VALUES (3, 'c')
+```
+4. A validation workflow can validate (e.g. data quality) the state of `audit-branch`.
+5. After validation, the main branch can be `fastForward` to the head of `audit-branch` to update the main table state.
+```java
+table.manageSnapshots().fastForward("main", "audit-branch").commit()
+```
+6. The branch reference will be removed when `expireSnapshots` is run 1 week later.
+
+## Usage 
+
+Creating, querying and writing to branches and tags are supported in the Iceberg Java library, and in Spark and Flink engine integrations.
+
+- [Iceberg Java Library](../../java-api-quickstart/#branching-and-tagging)
+- [Spark DDLs](../spark-ddl/#branching-and-tagging-ddl)
+- [Spark Reads](../spark-queries/#time-travel)
+- [Spark Branch Writes](../spark-writes/#writing-to-branches)
+- [Flink Reads](../flink-queries/#reading-branches-and-tags-with-SQL)
+- [Flink Branch Writes](../flink-writes/#branch-writes)
\ No newline at end of file
diff --git a/docs/content/delta-lake-migration.md b/docs/content/delta-lake-migration.md
new file mode 100644
index 00000000..2c1dddf5
--- /dev/null
+++ b/docs/content/delta-lake-migration.md
@@ -0,0 +1,122 @@
+---
+title: "Delta Lake Migration"
+url: delta-lake-migration
+menu:
+  main:
+    parent: "Migration"
+    weight: 300
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# Delta Lake Table Migration
+Delta Lake is a table format that supports Parquet file format and provides time travel and versioning features. When migrating data from Delta Lake to Iceberg,
+it is common to migrate all snapshots to maintain the history of the data.
+
+Currently, Iceberg supports the Snapshot Table action for migrating from Delta Lake to Iceberg tables.
+Since Delta Lake tables maintain transactions, all available transactions will be committed to the new Iceberg table as transactions in order.
+For Delta Lake tables, any additional data files added after the initial migration will be included in their corresponding transactions and subsequently added to the new Iceberg table using the Add Transaction action.
+The Add Transaction action, a variant of the Add File action, is still under development.
+
+## Enabling Migration from Delta Lake to Iceberg
+The `iceberg-delta-lake` module is not bundled with Spark and Flink engine runtimes. To enable migration from delta lake features, the minimum required dependencies are:
+- [iceberg-delta-lake](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-delta-lake/1.2.1/iceberg-delta-lake-1.2.1.jar)
+- [delta-standalone-0.6.0](https://repo1.maven.org/maven2/io/delta/delta-standalone_2.13/0.6.0/delta-standalone_2.13-0.6.0.jar)
+- [delta-storage-2.2.0](https://repo1.maven.org/maven2/io/delta/delta-storage/2.2.0/delta-storage-2.2.0.jar)
+
+### Compatibilities
+The module is built and tested with `Delta Standalone:0.6.0` and supports Delta Lake tables with the following protocol version:
+* `minReaderVersion`: 1
+* `minWriterVersion`: 2
+
+Please refer to [Delta Lake Table Protocol Versioning](https://docs.delta.io/latest/versioning.html) for more details about Delta Lake protocol versions.
+
+### API
+The `iceberg-delta-lake` module provides an interface named `DeltaLakeToIcebergMigrationActionsProvider`, which contains actions that helps converting from Delta Lake to Iceberg.
+The supported actions are:
+* `snapshotDeltaLakeTable`: snapshot an existing Delta Lake table to an Iceberg table
+
+### Default Implementation
+The `iceberg-delta-lake` module also provides a default implementation of the interface which can be accessed by
+```java
+DeltaLakeToIcebergMigrationActionsProvider defaultActions = DeltaLakeToIcebergMigrationActionsProvider.defaultActions()
+```
+
+## Snapshot Delta Lake Table to Iceberg
+The action `snapshotDeltaLakeTable` reads the Delta Lake table's transactions and converts them to a new Iceberg table with the same schema and partitioning in one iceberg transaction.
+The original Delta Lake table remains unchanged.
+
+The newly created table can be changed or written to without affecting the source table, but the snapshot uses the original table's data files.
+Existing data files are added to the Iceberg table's metadata and can be read using a name-to-id mapping created from the original table schema.
+
+When inserts or overwrites run on the snapshot, new files are placed in the snapshot table's location. The location is default to be the same as that
+of the source Delta Lake Table. Users can also specify a different location for the snapshot table.
+
+{{< hint info >}}
+Because tables created by `snapshotDeltaLakeTable` are not the sole owners of their data files, they are prohibited from
+actions like `expire_snapshots` which would physically delete data files. Iceberg deletes, which only effect metadata,
+are still allowed. In addition, any operations which affect the original data files will disrupt the Snapshot's
+integrity. DELETE statements executed against the original Delta Lake table will remove original data files and the
+`snapshotDeltaLakeTable` table will no longer be able to access them.
+{{< /hint >}}
+
+#### Usage
+| Required Input               | Configured By                                                                                                                                                                                             | Description                                                                     |
+|------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------|
+| Source Table Location        | Argument [`sourceTableLocation`](https://iceberg.apache.org/javadoc/latest/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.html#snapshotDeltaLakeTable(java.lang.String))             | The location of the source Delta Lake table                                     | 
+| New Iceberg Table Identifier | Configuration API [`as`](https://iceberg.apache.org/javadoc/latest/org/apache/iceberg/delta/SnapshotDeltaLakeTable.html#as(org.apache.iceberg.catalog.TableIdentifier))                                   | The identifier specifies the namespace and table name for the new iceberg table |
+| Iceberg Catalog              | Configuration API [`icebergCatalog`](https://iceberg.apache.org/javadoc/latest/org/apache/iceberg/delta/SnapshotDeltaLakeTable.html#icebergCatalog(org.apache.iceberg.catalog.Catalog))                   | The catalog used to create the new iceberg table                                |
+| Hadoop Configuration         | Configuration API [`deltaLakeConfiguration`](https://iceberg.apache.org/javadoc/latest/org/apache/iceberg/delta/SnapshotDeltaLakeTable.html#deltaLakeConfiguration(org.apache.hadoop.conf.Configuration)) | The Hadoop Configuration used to read the source Delta Lake table.              |
+
+For detailed usage and other optional configurations, please refer to the [SnapshotDeltaLakeTable API](https://iceberg.apache.org/javadoc/latest/org/apache/iceberg/delta/SnapshotDeltaLakeTable.html)
+
+#### Output
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| `imported_files_count` | long | Number of files added to the new table |
+
+#### Added Table Properties
+The following table properties are added to the Iceberg table to be created by default:
+
+| Property Name                 | Value                                     | Description                                                        |
+|-------------------------------|-------------------------------------------|--------------------------------------------------------------------|
+| `snapshot_source`             | `delta`                                   | Indicates that the table is snapshot from a delta lake table       |
+| `original_location`           | location of the delta lake table          | The absolute path to the location of the original delta lake table |
+| `schema.name-mapping.default` | JSON name mapping derived from the schema | The name mapping string used to read Delta Lake table's data files |
+
+#### Examples
+```java
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.delta.DeltaLakeToIcebergMigrationActionsProvider;
+
+String sourceDeltaLakeTableLocation = "s3://my-bucket/delta-table";
+String destTableLocation = "s3://my-bucket/iceberg-table";
+TableIdentifier destTableIdentifier = TableIdentifier.of("my_db", "my_table");
+Catalog icebergCatalog = ...; // Iceberg Catalog fetched from engines like Spark or created via CatalogUtil.loadCatalog
+Configuration hadoopConf = ...; // Hadoop Configuration fetched from engines like Spark and have proper file system configuration to access the Delta Lake table.
+    
+DeltaLakeToIcebergMigrationActionsProvider.defaultActions()
+    .snapshotDeltaLakeTable(sourceDeltaLakeTableLocation)
+    .as(destTableIdentifier)
+    .icebergCatalog(icebergCatalog)
+    .tableLocation(destTableLocation)
+    .deltaLakeConfiguration(hadoopConf)
+    .tableProperty("my_property", "my_value")
+    .execute();
+```
diff --git a/docs/content/flink-configuration.md b/docs/content/flink-configuration.md
index 7e531baa..89741159 100644
--- a/docs/content/flink-configuration.md
+++ b/docs/content/flink-configuration.md
@@ -118,7 +118,11 @@ env.getConfig()
 | 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.                                  |
+| 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. |
diff --git a/docs/content/flink-getting-started.md b/docs/content/flink-getting-started.md
index 677d628c..078bc124 100644
--- a/docs/content/flink-getting-started.md
+++ b/docs/content/flink-getting-started.md
@@ -271,6 +271,16 @@ FlinkSink.forRowData(input)
 env.execute("Test Iceberg DataStream");
 ```
 
+### Branch Writes
+Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink`
+For more information on branches please refer to [branches](../../tables/branching).
+```java
+FlinkSink.forRowData(input)
+    .tableLoader(tableLoader)
+    .toBranch("audit-branch")
+    .append();
+```
+
 ## Reading
 
 Submit a Flink __batch__ job using the following sentences:
diff --git a/docs/content/flink-queries.md b/docs/content/flink-queries.md
index 9afabe28..2d62d18e 100644
--- a/docs/content/flink-queries.md
+++ b/docs/content/flink-queries.md
@@ -80,6 +80,21 @@ Here are the SQL settings for the [FLIP-27](https://cwiki.apache.org/confluence/
 SET table.exec.iceberg.use-flip27-source = true;
 ```
 
+### Reading branches and tags with SQL
+Branch and tags can be read via SQL by specifying options. For more details
+refer to [Flink Configuration](../flink-configuration/#read-options)
+
+```sql
+--- Read from branch b1
+SELECT * FROM table /*+ OPTIONS('branch'='b1') */ ;
+
+--- Read from tag t1
+SELECT * FROM table /*+ OPTIONS('tag'='t1') */;
+
+--- Incremental scan from tag t1 to tag t2
+SELECT * FROM table /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='t1', 'end-tag'='t2') */;
+```
+
 ## Reading with DataStream
 
 Iceberg support streaming or batch read in Java API now.
@@ -197,6 +212,37 @@ env.execute("Test Iceberg Streaming Read");
 There are other options that could be set by Java API, please see the
 [IcebergSource#Builder](../../../javadoc/{{% icebergVersion %}}/org/apache/iceberg/flink/source/IcebergSource.html).
 
+### Reading branches and tags with DataStream
+Branches and tags can also be read via the DataStream API
+
+```java
+StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path");
+// Read from branch
+DataStream<RowData> batch = FlinkSource.forRowData()
+    .env(env)
+    .tableLoader(tableLoader)
+    .branch("test-branch")
+    .streaming(false)
+    .build();
+
+// Read from tag
+DataStream<RowData> batch = FlinkSource.forRowData()
+    .env(env)
+    .tableLoader(tableLoader)
+    .tag("test-tag")
+    .streaming(false)
+    .build();
+
+// Streaming read from start-tag
+DataStream<RowData> batch = FlinkSource.forRowData()
+    .env(env)
+    .tableLoader(tableLoader)
+    .streaming(true)
+    .startTag("test-tag")
+    .build();
+```
+
 ### Read as Avro GenericRecord
 
 FLIP-27 Iceberg source provides `AvroGenericRecordReaderFunction` that converts
diff --git a/docs/content/flink-writes.md b/docs/content/flink-writes.md
index 22cf0778..f61416bf 100644
--- a/docs/content/flink-writes.md
+++ b/docs/content/flink-writes.md
@@ -190,6 +190,16 @@ FlinkSink.builderFor(
   .append();
 ```
 
+### Branch Writes
+Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink`
+For more information on branches please refer to [branches](../../tables/branching).
+```java
+FlinkSink.forRowData(input)
+    .tableLoader(tableLoader)
+    .toBranch("audit-branch")
+    .append();
+```
+
 ### Metrics
 
 The following Flink metrics are provided by the Flink Iceberg sink.
diff --git a/docs/content/hive-migration.md b/docs/content/hive-migration.md
new file mode 100644
index 00000000..c4ef2562
--- /dev/null
+++ b/docs/content/hive-migration.md
@@ -0,0 +1,60 @@
+---
+title: "Hive Migration"
+url: hive-migration
+menu:
+  main:
+    parent: "Migration"
+    weight: 200
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# Hive Table Migration
+Apache Hive supports ORC, Parquet, and Avro file formats that could be migrated to Iceberg.
+When migrating data to an Iceberg table, which provides versioning and transactional updates, only the most recent data files need to be migrated.
+
+Iceberg supports all three migration actions: Snapshot Table, Migrate Table, and Add Files for migrating from Hive tables to Iceberg tables. Since Hive tables do not maintain snapshots,
+the migration process essentially involves creating a new Iceberg table with the existing schema and committing all data files across all partitions to the new Iceberg table.
+After the initial migration, any new data files are added to the new Iceberg table using the Add Files action.
+
+## Enabling Migration from Hive to Iceberg
+The Hive table migration actions are supported by the Spark Integration module via Spark Procedures. 
+The procedures are bundled in the Spark runtime jar, which is available in the [Iceberg Release Downloads](https://iceberg.apache.org/releases/#downloads).
+
+## Snapshot Hive Table to Iceberg
+To snapshot a Hive table, users can run the following Spark SQL:
+```sql
+CALL catalog_name.system.snapshot('db.source', 'db.dest')
+```
+See [Spark Procedure: snapshot](../spark-procedures/#snapshot) for more details.
+
+## Migrate Hive Table To Iceberg
+To migrate a Hive table to Iceberg, users can run the following Spark SQL:
+```sql
+CALL catalog_name.system.migrate('db.sample')
+```
+See [Spark Procedure: migrate](../spark-procedures/#migrate) for more details.
+
+## Add Files From Hive Table to Iceberg
+To add data files from a Hive table to a given Iceberg table, users can run the following Spark SQL:
+```sql
+CALL spark_catalog.system.add_files(
+table => 'db.tbl',
+source_table => 'db.src_tbl'
+)
+```
+See [Spark Procedure: add_files](../spark-procedures/#add_files) for more details.
diff --git a/docs/content/spark-ddl.md b/docs/content/spark-ddl.md
index 2f979b54..9eaae534 100644
--- a/docs/content/spark-ddl.md
+++ b/docs/content/spark-ddl.md
@@ -471,4 +471,64 @@ ALTER TABLE prod.db.sample DROP IDENTIFIER FIELDS id, data
 -- multiple columns
 ```
 
-Note that although the identifier is removed, the column will still exist in the table schema.
\ No newline at end of file
+Note that although the identifier is removed, the column will still exist in the table schema.
+
+### Branching and Tagging DDL
+
+#### `ALTER TABLE ... CREATE BRANCH`
+
+Branches can be created via the `CREATE BRANCH` statement, which includes 
+the snapshot to create the branch at and an optional retention clause.
+
+```sql
+-- CREATE audit-branch at snapshot 1234 with default retention.
+ALTER TABLE prod.db.sample CREATE BRANCH audit-branch
+AS OF VERSION 1234
+
+-- CREATE audit-branch at snapshot 1234, retain audit-branch for 31 days, and retain the latest 31 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots 
+ALTER TABLE prod.db.sample CREATE BRANCH audit-branch
+AS OF VERSION 1234 RETAIN 30 DAYS 
+WITH RETENTION 3 SNAPSHOTS 2 DAYS
+```
+
+
+#### `ALTER TABLE ... CREATE TAG`
+
+Tags can be created via the `CREATE TAG` statement, which includes 
+the snapshot to create the branch at and an optional retention clause.
+
+```sql
+-- CREATE historical-tag at snapshot 1234 with default retention.
+ALTER TABLE prod.db.sample CREATE TAG historical-tag AS OF VERSION 1234
+
+-- CREATE historical-tag at snapshot 1234 and retain it for 1 year. 
+ALTER TABLE prod.db.sample CREATE TAG historical-tag 
+AS OF VERSION 1234 RETAIN 365 DAYS
+```
+
+### `ALTER TABLE ... REPLACE BRANCH`
+
+The snapshot which a branch references can be updated via
+the `REPLACE BRANCH` sql. Retention can also be updated in this statement. 
+
+```sql
+-- REPLACE audit-branch to reference snapshot 4567 and update the retention to 60 days
+ALTER TABLE prod.db.sample REPLACE BRANCH audit-branch
+AS OF VERSION 4567 RETAIN 60 DAYS
+```
+
+#### `ALTER TABLE ... DROP BRANCH`
+
+Branches can be removed via the `DROP BRANCH` sql
+
+```sql
+ALTER TABLE prod.db.sample DROP BRANCH audit-branch
+```
+
+#### `ALTER TABLE ... DROP TAG`
+
+Tags can be removed via the `DROP TAG` sql
+
+```sql
+ALTER TABLE prod.db.sample DROP TAG historical-tag
+```
\ No newline at end of file
diff --git a/docs/content/spark-writes.md b/docs/content/spark-writes.md
index 08f788fe..484aee8f 100644
--- a/docs/content/spark-writes.md
+++ b/docs/content/spark-writes.md
@@ -203,6 +203,36 @@ WHERE EXISTS (SELECT oid FROM prod.db.returned_orders WHERE t1.oid = oid)
 
 For more complex row-level updates based on incoming data, see the section on `MERGE INTO`.
 
+## Writing to Branches
+Branch writes can be performed via SQL by providing a branch identifier, `branch_yourBranch` in the operation.
+Branch writes can also be performed as part of a write-audit-publish (WAP) workflow by specifying the `spark.wap.branch` config.
+Note WAP branch and branch identifier cannot both be specified.
+Also, the branch must exist before performing the write. 
+The operation does **not** create the branch if it does not exist. 
+For more information on branches please refer to [branches](../../tables/branching)
+ 
+```sql
+-- INSERT (1,' a') (2, 'b') into the audit branch.
+INSERT INTO prod.db.table.branch_audit VALUES (1, 'a'), (2, 'b');
+
+-- MERGE INTO audit branch
+MERGE INTO prod.db.table.branch_audit t 
+USING (SELECT ...) s        
+ON t.id = s.id          
+WHEN ...
+
+-- UPDATE audit branch
+UPDATE prod.db.table.branch_audit AS t1
+SET val = 'c'
+
+-- DELETE FROM audit branch
+DELETE FROM prod.dbl.table.branch_audit WHERE id = 2;
+
+-- WAP Branch write
+SET spark.wap.branch = audit-branch
+INSERT INTO prod.db.table VALUES (3, 'c');
+```
+
 ## Writing with DataFrames
 
 Spark 3 introduced the new `DataFrameWriterV2` API for writing to tables using data frames. The v2 API is recommended for several reasons:
diff --git a/docs/content/table-migration.md b/docs/content/table-migration.md
new file mode 100644
index 00000000..9635a643
--- /dev/null
+++ b/docs/content/table-migration.md
@@ -0,0 +1,79 @@
+---
+title: "Overview"
+url: table-migration
+menu:
+  main:
+    parent: "Migration"
+    weight: 100
+---
+<!--
+ - Licensed to the Apache Software Foundation (ASF) under one or more
+ - contributor license agreements.  See the NOTICE file distributed with
+ - this work for additional information regarding copyright ownership.
+ - The ASF licenses this file to You under the Apache License, Version 2.0
+ - (the "License"); you may not use this file except in compliance with
+ - the License.  You may obtain a copy of the License at
+ -
+ -   http://www.apache.org/licenses/LICENSE-2.0
+ -
+ - Unless required by applicable law or agreed to in writing, software
+ - distributed under the License is distributed on an "AS IS" BASIS,
+ - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ - See the License for the specific language governing permissions and
+ - limitations under the License.
+ -->
+
+# Table Migration
+Apache Iceberg supports converting existing tables in other formats to Iceberg tables. This section introduces the general concept of table migration, its approaches, and existing implementations in Iceberg.
+
+## Migration Approaches
+There are two methods for executing table migration: full data migration and in-place metadata migration.
+
+Full data migration involves copying all data files from the source table to the new Iceberg table. This method makes the new table fully isolated from the source table, but is slower and doubles the space.
+In practice, users can use operations like [Create-Table-As-Select](../spark-ddl/#create-table--as-select), [INSERT](../spark-writes/#insert-into), and Change-Data-Capture pipelines to perform such migration.
+
+In-place metadata migration preserves the existing data files while incorporating Iceberg metadata on top of them.
+This method is not only faster but also eliminates the need for data duplication. However, the new table and the source table are not fully isolated. In other words, if any processes vacuum data files from the source table, the new table will also be affected.
+
+In this doc, we will describe more about in-place metadata migration.
+
+![In-Place Metadata Migration](../../../img/iceberg-in-place-metadata-migration.png)
+
+Apache Iceberg supports the in-place metadata migration approach, which includes three important actions: **Snapshot Table**, **Migrate Table**, and **Add Files**.
+
+## Snapshot Table
+The Snapshot Table action creates a new iceberg table with a different name and with the same schema and partitioning as the source table, leaving the source table unchanged during and after the action.
+
+- Create a new Iceberg table with the same metadata (schema, partition spec, etc.) as the source table and a different name. Readers and Writers on the source table can continue to work.
+
+![Snapshot Table Step 1](../../../img/iceberg-snapshotaction-step1.png)
+
+- Commit all data files across all partitions to the new Iceberg table. The source table remains unchanged. Readers can be switched to the new Iceberg table.
+
+![Snapshot Table Step 2](../../../img/iceberg-snapshotaction-step2.png)
+
+- Eventually, all writers can be switched to the new Iceberg table. Once all writers are transitioned to the new Iceberg table, the migration process will be considered complete.
+
+## Migrate Table
+The Migrate Table action also creates a new Iceberg table with the same schema and partitioning as the source table. However, during the action execution, it locks and drops the source table from the catalog.
+Consequently, Migrate Table requires all modifications working on the source table to be stopped before the action is performed.
+
+Stop all writers interacting with the source table. Readers that also support Iceberg may continue reading.
+
+![Migrate Table Step 1](../../../img/iceberg-migrateaction-step1.png)
+
+- Create a new Iceberg table with the same identifier and metadata (schema, partition spec, etc.) as the source table. Rename the source table for a backup in case of failure and rollback.
+
+![Migrate Table Step 2](../../../img/iceberg-migrateaction-step2.png)
+
+- Commit all data files across all partitions to the new Iceberg table. Drop the source table. Writers can start writing to the new Iceberg table.
+
+![Migrate Table Step 3](../../../img/iceberg-migrateaction-step3.png)
+
+## Add Files
+After the initial step (either Snapshot Table or Migrate Table), it is common to find some data files that have not been migrated. These files often originate from concurrent writers who continue writing to the source table during or after the migration process.
+In practice, these files can be new data files in Hive tables or new snapshots (versions) of Delta Lake tables. The Add Files action is essential for incorporating these files into the Iceberg table.
+
+# Migrating From Different Table Formats
+* [From Hive to Iceberg](../hive-migration)
+* [From Delta Lake to Iceberg](../delta-lake-migration)