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

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


The following commit(s) were added to refs/heads/main by this push:
     new 9c5cc5b2 Copy latest docs from iceberg repo into iceberg-docs (#228)
9c5cc5b2 is described below

commit 9c5cc5b241c177f153e5e2006eff1b8597c0c097
Author: Amogh Jahagirdar <[email protected]>
AuthorDate: Sat Apr 29 14:32:36 2023 -0700

    Copy latest docs from iceberg repo into iceberg-docs (#228)
---
 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/hive.md                       |   4 +-
 docs/content/java-api-quickstart.md        | 124 +++++++++++++++++++++++++++++
 docs/content/spark-configuration.md        |  10 ---
 docs/content/spark-ddl.md                  |  72 ++++++++++++++---
 docs/content/spark-queries.md              |  46 +----------
 docs/content/spark-structured-streaming.md |   4 -
 docs/content/spark-writes.md               |  78 +++++++++---------
 docs/content/table-migration.md            |  79 ++++++++++++++++++
 15 files changed, 684 insertions(+), 109 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/hive.md b/docs/content/hive.md
index 2e9eb217..d1138572 100644
--- a/docs/content/hive.md
+++ b/docs/content/hive.md
@@ -127,12 +127,10 @@ The table level configuration overrides the global Hadoop 
configuration.
 
 To use the Tez engine on Hive `3.1.2` or later, Tez needs to be upgraded to >= 
`0.10.1` which contains a necessary fix 
[TEZ-4248](https://issues.apache.org/jira/browse/TEZ-4248).
 
-
 To use the Tez engine on Hive `2.3.x`, you will need to manually build Tez 
from the `branch-0.9` branch due to a
 backwards incompatibility issue with Tez `0.10.1`.
 
-You will also need to set the following property in the Hive
-configuration: 
`tez.mrreader.config.update.properties=hive.io.file.readcolumn.names,hive.io.file.readcolumn.ids`.
+In both cases, you will also need to set the following property in the 
`tez-site.xml` configuration file: 
`tez.mrreader.config.update.properties=hive.io.file.readcolumn.names,hive.io.file.readcolumn.ids`.
 
 ## Catalog Management
 
diff --git a/docs/content/java-api-quickstart.md 
b/docs/content/java-api-quickstart.md
index 80769be3..24b61b42 100644
--- a/docs/content/java-api-quickstart.md
+++ b/docs/content/java-api-quickstart.md
@@ -198,3 +198,127 @@ PartitionSpec spec = PartitionSpec.builderFor(schema)
 ```
 
 For more information on the different partition transforms that Iceberg 
offers, visit [this page](../../../spec#partitioning).
+
+## Branching and Tagging
+
+### Creating branches and tags
+
+New branches and tags can be created via the Java library's ManageSnapshots 
API. 
+
+```java
+
+/* Create a branch test-branch which is retained for 1 week, and the latest 2 
snapshots on test-branch will always be retained. 
+Snapshots on test-branch which are created within the last hour will also be 
retained. */
+
+String branch = "test-branch";
+table.manageSnapshots()
+    .createBranch(branch, 3)
+    .setMinSnapshotsToKeep(branch, 2)
+    .setMaxSnapshotAgeMs(branch, 3600000)
+    .setMaxRefAgeMs(branch, 604800000)
+    .commit();
+
+// Create a tag historical-tag at snapshot 10 which is retained for a day
+String tag = "historical-tag"
+table.manageSnapshots()
+    .createTag(tag, 10)
+    .setMaxRefAgeMs(tag, 86400000)
+    .commit();
+```
+
+### Committing to branches
+
+Writing to a branch can be performed by specifying `toBranch` in the 
operation. For the full list refer to 
[UpdateOperations](../../java/api/#update-operations). 
+```java
+// Append FILE_A to branch test-branch 
+String branch = "test-branch";
+
+table.newAppend()
+    .appendFile(FILE_A)
+    .toBranch(branch)
+    .commit();
+
+
+// Perform row level updates on "test-branch"
+table.newRowDelta()
+    .addRows(DATA_FILE)
+    .addDeletes(DELETES)
+    .toBranch(branch)
+    .commit();
+
+
+// Perform a rewrite operation replacing small_file_1 and small_file_2 on 
"test-branch" with compacted_file.
+table.newRewrite()
+    .rewriteFiles(ImmutableSet.of(small_file_1,small_file_2), 
ImmutableSet.of(compacted_file))
+    .toBranch(branch)
+    .commit();
+
+```
+
+### Reading from branches and tags
+Reading from a branch or tag can be done as usual via the Table Scan API, by 
passing in a branch or tag in the `useRef` API. When a branch is passed in, the 
snapshot that's used is the head of the branch. Note that currently reading 
from a branch and specifying an `asOfSnapshotId` in the scan is not supported. 
+
+```java
+// Read from the head snapshot of test-branch
+TableScan branchRead = table.newScan().useRef("test-branch");
+
+// Read from the snapshot referenced by audit-tag
+Table tagRead = table.newScan().useRef("audit-tag");
+```
+
+### Replacing and fast forwarding branches and tags
+
+The snapshots which existing branches and tags point to can be updated via the 
`replace` APIs. The fast forward operation is similar to git fast-forwarding. 
Fast forward can be used to advance a target branch to the head of a source 
branch or a tag when the target branch is an ancestor of the source. For both 
fast forward and replace, retention properties of the target branch are 
maintained by default.
+
+```java
+
+// Update "test-branch" to point to snapshot 4
+table.manageSnapshots()
+     .replaceBranch(branch, 4)
+     .commit()
+
+String tag = "audit-tag";
+// Replace "audit-tag" to point to snapshot 3 and update its retention
+table.manageSnapshots()
+     .replaceBranch(tag, 4)
+     .setMaxRefAgeMs(1000)
+     .commit()
+
+
+```
+
+### Updating retention properties
+
+Retention properties for branches and tags can be updated as well.
+Use the setMaxRefAgeMs for updating the retention property of the branch or 
tag itself. Branch snapshot retention properties can be updated via the 
`setMinSnapshotsToKeep` and `setMaxSnapshotAgeMs` APIs. 
+
+```java
+String branch = "test-branch";
+// Update retention properties for test-branch
+table.manageSnapshots()
+    .setMinSnapshotsToKeep(branch, 10)
+    .setMaxSnapshotAgeMs(branch, 7200000)
+    .setMaxRefAgeMs(branch, 604800000)
+    .commit();
+
+// Update retention properties for test-tag
+table.manageSnapshots()
+    .setMaxRefAgeMs("test-tag", 604800000)
+    .commit();
+```
+
+### Removing branches and tags
+
+Branches and tags can be removed via the `removeBranch` and `removeTag` APIs 
respectively
+
+```java
+// Remove test-branch
+table.manageSnapshots()
+     .removeBranch("test-branch")
+     .commit()
+
+// Remove test-tag
+table.manageSnapshots()
+     .removeTag("test-tag")
+     .commit()
+```
diff --git a/docs/content/spark-configuration.md 
b/docs/content/spark-configuration.md
index 70c415db..926ec020 100644
--- a/docs/content/spark-configuration.md
+++ b/docs/content/spark-configuration.md
@@ -124,13 +124,6 @@ spark.sql.catalog.custom_prod.catalog-impl = 
com.my.custom.CatalogImpl
 spark.sql.catalog.custom_prod.my-additional-catalog-config = my-value
 ```
 
-### Catalogs in Spark 2.4
-
-When using Iceberg 0.11.0 and later, Spark 2.4 can load tables from multiple 
Iceberg catalogs or from table locations.
-
-Catalogs in 2.4 are configured just like catalogs in 3.x, but only Iceberg 
catalogs are supported.
-
-
 ## SQL Extensions
 
 Iceberg 0.11.0 and later add an extension module to Spark to add new SQL 
commands, like `CALL` for stored procedures or `ALTER TABLE ... WRITE ORDERED 
BY`.
@@ -142,9 +135,6 @@ Using those SQL commands requires adding Iceberg extensions 
to your Spark enviro
 
|---------------------------|---------------------------------------------------------------------|
 | `spark.sql.extensions`    | 
`org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions` |
 
-SQL extensions are not available for Spark 2.4.
-
-
 ## Runtime configuration
 
 ### Read options
diff --git a/docs/content/spark-ddl.md b/docs/content/spark-ddl.md
index 2f979b54..b8622856 100644
--- a/docs/content/spark-ddl.md
+++ b/docs/content/spark-ddl.md
@@ -27,13 +27,7 @@ menu:
 
 # Spark DDL
 
-To use Iceberg in Spark, first configure [Spark 
catalogs](../spark-configuration).
-
-Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog 
implementations. Spark DSv2 is an evolving API with different levels of support 
in Spark versions. Spark 2.4 does not support SQL DDL.
-
-{{< hint info >}}
-Spark 2.4 can't create Iceberg tables with DDL, instead use Spark 3 or the 
[Iceberg API](..//java-api-quickstart).
-{{< /hint >}}
+To use Iceberg in Spark, first configure [Spark 
catalogs](../spark-configuration). Iceberg uses Apache Spark's DataSourceV2 API 
for data source and catalog implementations.
 
 ## `CREATE TABLE`
 
@@ -256,7 +250,7 @@ ADD COLUMN points.value.b int
 
 Note: Altering a map 'key' column by adding columns is not allowed. Only map 
values can be updated.
 
-In Spark 2.4.4 and later, you can add columns in any position by adding 
`FIRST` or `AFTER` clauses:
+Add columns in any position by adding `FIRST` or `AFTER` clauses:
 
 ```sql
 ALTER TABLE prod.db.sample
@@ -471,4 +465,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-queries.md b/docs/content/spark-queries.md
index 7d5ec606..f2ebf1d8 100644
--- a/docs/content/spark-queries.md
+++ b/docs/content/spark-queries.md
@@ -27,22 +27,7 @@ menu:
 
 # Spark Queries
 
-To use Iceberg in Spark, first configure [Spark 
catalogs](../spark-configuration).
-
-Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog 
implementations. Spark DSv2 is an evolving API with different levels of support 
in Spark versions:
-
-| Feature support                                  | Spark 3 | Spark 2.4  | 
Notes                                          |
-|--------------------------------------------------|-----------|------------|------------------------------------------------|
-| [`SELECT`](#querying-with-sql)                   | ✔️        |            |  
                                              |
-| [DataFrame reads](#querying-with-dataframes)     | ✔️        | ✔️          | 
                                               |
-| [Metadata table `SELECT`](#inspecting-tables)    | ✔️        |            |  
                                              |
-| [History metadata table](#history)               | ✔️        | ✔️          | 
                                               |
-| [Snapshots metadata table](#snapshots)           | ✔️        | ✔️          | 
                                               |
-| [Files metadata table](#files)                   | ✔️        | ✔️          | 
                                               |
-| [Manifests metadata table](#manifests)           | ✔️        | ✔️          | 
                                               |
-| [Partitions metadata table](#partitions)         | ✔️        | ✔️          | 
                                               |
-| [All metadata tables](#all-metadata-tables)      | ✔️        | ✔️          | 
                                               |
-
+To use Iceberg in Spark, first configure [Spark 
catalogs](../spark-configuration). Iceberg uses Apache Spark's DataSourceV2 API 
for data source and catalog implementations.
 
 ## Querying with SQL
 
@@ -75,8 +60,6 @@ val df = spark.table("prod.db.table")
 
 ### Catalogs with DataFrameReader
 
-Iceberg 0.11.0 adds multi-catalog support to `DataFrameReader` in both Spark 3 
and 2.4.
-
 Paths and table names can be loaded with Spark's `DataFrameReader` interface. 
How tables are loaded depends on how
 the identifier is specified. When using 
`spark.read.format("iceberg").load(table)` or `spark.table(table)` the `table`
 variable can take a number of forms as listed below:
@@ -205,29 +188,6 @@ Incremental read works with both V1 and V2 format-version.
 Incremental read is not supported by Spark's SQL syntax.
 {{< /hint >}}
 
-### Spark 2.4
-
-Spark 2.4 requires using the DataFrame reader with `iceberg` as a format, 
because 2.4 does not support direct SQL queries:
-
-```scala
-// named metastore table
-spark.read.format("iceberg").load("catalog.db.table")
-// Hadoop path table
-spark.read.format("iceberg").load("hdfs://nn:8020/path/to/table")
-```
-
-#### Spark 2.4 with SQL
-
-To run SQL `SELECT` statements on Iceberg tables in 2.4, register the 
DataFrame as a temporary table:
-
-```scala
-val df = spark.read.format("iceberg").load("db.table")
-df.createOrReplaceTempView("table")
-
-spark.sql("""select count(1) from table""").show()
-```
-
-
 ## Inspecting tables
 
 To inspect a table's history, snapshots, and other metadata, Iceberg supports 
metadata tables.
@@ -235,8 +195,6 @@ To inspect a table's history, snapshots, and other 
metadata, Iceberg supports me
 Metadata tables are identified by adding the metadata table name after the 
original table name. For example, history for `db.table` is read using 
`db.table.history`.
 
 {{< hint info >}}
-For Spark 2.4, use the `DataFrameReader` API to [inspect 
tables](#inspecting-with-dataframes).
-
 For Spark 3, prior to 3.2, the Spark [session 
catalog](../spark-configuration#replacing-the-session-catalog) does not support 
table names with multipart identifiers such as 
`catalog.database.table.metadata`. As a workaround, configure an 
`org.apache.iceberg.spark.SparkCatalog`, or use the Spark `DataFrameReader` API.
 {{< /hint >}}
 
@@ -422,7 +380,7 @@ SELECT * FROM prod.db.table.refs;
 
 ### Inspecting with DataFrames
 
-Metadata tables can be loaded in Spark 2.4 or Spark 3 using the 
DataFrameReader API:
+Metadata tables can be loaded using the DataFrameReader API:
 
 ```scala
 // named metastore table
diff --git a/docs/content/spark-structured-streaming.md 
b/docs/content/spark-structured-streaming.md
index bdb4b340..77a79608 100644
--- a/docs/content/spark-structured-streaming.md
+++ b/docs/content/spark-structured-streaming.md
@@ -32,10 +32,6 @@ with different levels of support in Spark versions.
 
 As of Spark 3, DataFrame reads and writes are supported.
 
-| Feature support                                  | Spark 3 | Spark 2.4  | 
Notes                                          |
-|--------------------------------------------------|-----------|------------|------------------------------------------------|
-| [DataFrame write](#streaming-writes)             | ✔         | ✔          |  
                                              |
-
 ## Streaming Reads
 
 Iceberg supports processing incremental data in spark structured streaming 
jobs which starts from a historical timestamp:
diff --git a/docs/content/spark-writes.md b/docs/content/spark-writes.md
index 08f788fe..87a58452 100644
--- a/docs/content/spark-writes.md
+++ b/docs/content/spark-writes.md
@@ -33,16 +33,16 @@ Some plans are only available when using [Iceberg SQL 
extensions](../spark-confi
 
 Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog 
implementations. Spark DSv2 is an evolving API with different levels of support 
in Spark versions:
 
-| Feature support                                  | Spark 3 | Spark 2.4  | 
Notes                                        |
-|--------------------------------------------------|-----------|------------|----------------------------------------------|
-| [SQL insert into](#insert-into)                  | ✔️        |            |  
                                            |
-| [SQL merge into](#merge-into)                    | ✔️        |            | 
⚠ Requires Iceberg Spark extensions          |
-| [SQL insert overwrite](#insert-overwrite)        | ✔️        |            |  
                                            |
-| [SQL delete from](#delete-from)                  | ✔️        |            | 
⚠ Row-level delete requires Spark extensions |
-| [SQL update](#update)                            | ✔️        |            | 
⚠ Requires Iceberg Spark extensions          |
-| [DataFrame append](#appending-data)              | ✔️        | ✔️          | 
                                             |
-| [DataFrame overwrite](#overwriting-data)         | ✔️        | ✔️          | 
⚠ Behavior changed in Spark 3              |
-| [DataFrame CTAS and RTAS](#creating-tables)      | ✔️        |            |  
                                            |
+| Feature support                                  | Spark 3 | Notes           
                             |
+|--------------------------------------------------|-----------|----------------------------------------------|
+| [SQL insert into](#insert-into)                  | ✔️        |               
                               |
+| [SQL merge into](#merge-into)                    | ✔️        | ⚠ Requires 
Iceberg Spark extensions          |
+| [SQL insert overwrite](#insert-overwrite)        | ✔️        |               
                               |
+| [SQL delete from](#delete-from)                  | ✔️        | ⚠ Row-level 
delete requires Spark extensions |
+| [SQL update](#update)                            | ✔️        | ⚠ Requires 
Iceberg Spark extensions          |
+| [DataFrame append](#appending-data)              | ✔️        |               
                               |
+| [DataFrame overwrite](#overwriting-data)         | ✔️        |               
                               |
+| [DataFrame CTAS and RTAS](#creating-tables)      | ✔️        |               
                               |
 
 
 ## Writing with SQL
@@ -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:
@@ -234,17 +264,6 @@ val data: DataFrame = ...
 data.writeTo("prod.db.table").append()
 ```
 
-#### Spark 2.4
-
-In Spark 2.4, use the v1 API with `append` mode and `iceberg` format:
-
-```scala
-data.write
-    .format("iceberg")
-    .mode("append")
-    .save("db.table")
-```
-
 ### Overwriting data
 
 To overwrite partitions dynamically, use `overwritePartitions()`:
@@ -260,23 +279,6 @@ To explicitly overwrite partitions, use `overwrite` to 
supply a filter:
 data.writeTo("prod.db.table").overwrite($"level" === "INFO")
 ```
 
-#### Spark 2.4
-
-In Spark 2.4, overwrite values in an Iceberg table with `overwrite` mode and 
`iceberg` format:
-
-```scala
-data.write
-    .format("iceberg")
-    .mode("overwrite")
-    .save("db.table")
-```
-
-{{< hint danger >}}
-**The behavior of overwrite mode changed between Spark 2.4 and Spark 3**.
-{{< /hint >}}
-
-The behavior of DataFrameWriter overwrite mode was undefined in Spark 2.4, but 
is required to overwrite the entire table in Spark 3. Because of this new 
requirement, the Iceberg source's behavior changed in Spark 3. In Spark 2.4, 
the behavior was to dynamically overwrite partitions. To use the Spark 2.4 
behavior, add option `overwrite-mode=dynamic`.
-
 ### Creating tables
 
 To run a CTAS or RTAS, use `create`, `replace`, or `createOrReplace` 
operations:
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)

Reply via email to