You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/01/11 20:36:00 UTC

[GitHub] [iceberg] RussellSpitzer opened a new pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

RussellSpitzer opened a new pull request #2067:
URL: https://github.com/apache/iceberg/pull/2067


   


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |

Review comment:
       If we do this we need to change it in all the other places as well. I'd like to keep the usage the same as the function, but if we do this swap we should do it everywhere




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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |

Review comment:
       OK, yes, makes sense to reword to keep "ago" (or some other indication that it's older than this point in time).




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   @massdosage Thanks for all your feedback, I accepted a large amount of it and did some other modifications to address some of the remaining points. Please let me know if you have any more suggestions.


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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see

Review comment:
       OK, it's just that "rollbacks" isn't a real word :). Perhaps you can say "Performs a rollback" instead to keep the function name?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   @aokolnychyi + @rdblue  Some docs for stored procedures


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )

Review comment:
       Probably too late? I'm not sure we actually did a release with this yet ...




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,329 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the Iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.

Review comment:
       How about "namespace" instead of "keyspace"?




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   I'm going to go head and merge this because I'm planning to do a lot of editing today. I'll make a few changes to this as well.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see

Review comment:
       this is another one where the function name uses "rollback" so I think we should keep that the same in the documentation.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )

Review comment:
       Looking in our API, we always use cherrypick as a single word. 
   
   https://github.com/RussellSpitzer/iceberg/blob/a0ff334f2a9b0e1a429ccf69a85a12e8c255ed33/api/src/main/java/org/apache/iceberg/ManageSnapshots.java#L82
   
   https://github.com/RussellSpitzer/iceberg/blob/a0ff334f2a9b0e1a429ccf69a85a12e8c255ed33/api/src/main/java/org/apache/iceberg/exceptions/CherrypickAncestorCommitException.java#L33
   
   
   But we also have a few usages in messages with it as two words, but a single word is most common.
   
   https://github.com/RussellSpitzer/iceberg/blob/c13ae6ce83351c04f384ca154d4daffa8272c1a9/core/src/main/java/org/apache/iceberg/SnapshotManager.java#L68
   
   Anyway I"ll leave changing this to someone else if they want to do a project wide swap
   




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,329 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the Iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.

Review comment:
       This should note that only appends and dynamic overwrites can be cherry-picked.




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

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



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


[GitHub] [iceberg] rdblue merged pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to now()) |

Review comment:
       This defaults to now? I thought it defaulted to 3 days ago?




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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will no longer be able to access them.
+
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |

Review comment:
       Ah, OK, thanks for the clarification. Maybe "Source table for the Iceberg table" then?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |

Review comment:
       Maybe I need to reword this whole bit, because I don't think the grammar makes sense if we drop "ago" because we pass a point in time here, not an age of the files.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all

Review comment:
       It's hard to word this. It is basically like "Create Table As SELECT" but you don't do a select, it uses the original files. So we make a new Iceberg table, but it refers to the files that were in this other source table at the time.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   Thanks @jackye1995 ! Changes made 


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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see

Review comment:
       OK, it's just that "rollbacks: isn't a word, perhaps you can say "Performs a rollback" instead to keep the function name?




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,329 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the Iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure

Review comment:
       While it is one word in the API, I do think that it should be hyphenated here: "Cherry-pick"




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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )

Review comment:
       I'm just reviewing this now and I think it's too late to change. Looks like it's one word now.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample

Review comment:
       without setting any new properites, I'll make the change because it will preserve original table properties. 




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

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



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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to

Review comment:
       nit: the use of

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |

Review comment:
       nit: to roll back

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to roll back to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This is basically a test version of
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |
+| table         | ✔️  | String | Name of the new Iceberg table to create |
+| location      |     | String | Table location for new table (Defaults to catalog's default location) |
+| properties    | ️   | Map<String, String> | Properties to add to the newly created table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| imported_files_count | Long | Number of files referenced by the new table |
+
+#### Examples
+
+Make an isolated Iceberg table which references table `db.sample` named `db.snap` at the
+catalog's default location for `db.snap`.
+```sql
+    CALL catalog_name hive_prod.system.snapshot('db.sample', 'db.snap')
+```
+
+Migrate an isolated Iceberg table which references table `db.sample` named `db.snap` at
+a manually specified location `/tmp/temptable/`.

Review comment:
       This location is not a valid URI. In this case, will it choose the URI scheme of the source table?

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to roll back to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This is basically a test version of

Review comment:
       I think it confuses the readers to say it is `a test version`. It is clearer to describe a few example use cases that a user should choose this procedure instead of the migrate table procedure.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will no longer be able to access them.
+
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |

Review comment:
       This doesn't have to be a Hive table, it can also work off any Spark v1 table in the session catalog




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |

Review comment:
       The wording here is again difficult for me, "ago" is necessary because the default is a point in time, not an age.




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

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will not longer be able to access them.

Review comment:
       nit: will no longer




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to roll back to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This is basically a test version of

Review comment:
       Of course, that would be very helpful




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to roll back to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This is basically a test version of
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |
+| table         | ✔️  | String | Name of the new Iceberg table to create |
+| location      |     | String | Table location for new table (Defaults to catalog's default location) |
+| properties    | ️   | Map<String, String> | Properties to add to the newly created table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| imported_files_count | Long | Number of files referenced by the new table |
+
+#### Examples
+
+Make an isolated Iceberg table which references table `db.sample` named `db.snap` at the
+catalog's default location for `db.snap`.
+```sql
+    CALL catalog_name hive_prod.system.snapshot('db.sample', 'db.snap')
+```
+
+Migrate an isolated Iceberg table which references table `db.sample` named `db.snap` at
+a manually specified location `/tmp/temptable/`.

Review comment:
       No, it basically sets the Iceberg Table location. 
   ```
   Catalog.TableBuilder.withLocation("/tmp/temptable/")
   ```
   Which will set the "location" parameter in TableMetadata, I think normally this is going to be resolved as a HDFS path so it would be the default scheme ('default.fs'), but I think it may be possible for a custom FileIO to resolve it differently.




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

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)

Review comment:
       should this be `CALL hive_prod.system.` instead of `CALL catalog_name hive_prod.system.`




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to now()) |

Review comment:
       I was totally wrong it's 5 days ...
   ```  public static final long MAX_SNAPSHOT_AGE_MS_DEFAULT = 5 * 24 * 60 * 60 * 1000; // 5 days
   ```
   

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to now()) |

Review comment:
       I was totally wrong it's 5 days ...
   ```  
   public static final long MAX_SNAPSHOT_AGE_MS_DEFAULT = 5 * 24 * 60 * 60 * 1000; // 5 days
   ```
   




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |

Review comment:
       Ah thanks, I'm going to next line's "rollback" it to one word though, because we use that everywhere else (and in the procedure name




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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


   @aokolnychyi  Any comments?


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

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)

Review comment:
       Sorry, I missed this, but I do see a couple more of these in the doc. Otherwise the doc LGTM!




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

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



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


[GitHub] [iceberg] jackye1995 commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to roll back to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This is basically a test version of
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |
+| table         | ✔️  | String | Name of the new Iceberg table to create |
+| location      |     | String | Table location for new table (Defaults to catalog's default location) |
+| properties    | ️   | Map<String, String> | Properties to add to the newly created table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| imported_files_count | Long | Number of files referenced by the new table |
+
+#### Examples
+
+Make an isolated Iceberg table which references table `db.sample` named `db.snap` at the
+catalog's default location for `db.snap`.
+```sql
+    CALL catalog_name hive_prod.system.snapshot('db.sample', 'db.snap')
+```
+
+Migrate an isolated Iceberg table which references table `db.sample` named `db.snap` at
+a manually specified location `/tmp/temptable/`.

Review comment:
       I see. In this case I think it is preferable to use a URI in the examples, because not all FileIO can resolve a non-URI path correctly.




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,309 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to now()) |

Review comment:
       I can go back and double check this, but Remove Orphans was 3 days ago and I think this was now




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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )

Review comment:
       OK, yeah, I see, probably best to do a global search and replace to be either two words or one across the whole project. My vote goes for two words (also for the method name( but up to the main project committers I guess. @rdblue ?




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

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



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


[GitHub] [iceberg] RussellSpitzer commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|

Review comment:
       Yep, i'll change that




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

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #2067: Adds initial Documentation for Iceberg Stored Procedures

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



##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to

Review comment:
       ```suggestion
   To call an Iceberg stored procedure, execute a `CALL` command against the Iceberg catalog. All procedures are added to
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for

Review comment:
       ```suggestion
   In Spark 3 Iceberg provides an SQL API for accomplishing [maintenance actions](maintenance.md). Support for
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |

Review comment:
       ```suggestion
   | table         | ✔️  | String | Name of table to perform the cherry pick on |
   ```
   "cherry-pick" would also be an alternative

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args

Review comment:
       ```suggestion
   Cherry pick Snapshot 1 with named args
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking

Review comment:
       ```suggestion
   Converts a table known to Spark in this catalog into an Iceberg table. This is mainly for taking
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample

Review comment:
       ```suggestion
   Migrate the table `db.sample` to an Iceberg table without setting any properties.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |

Review comment:
       ```suggestion
   | table         | ✔️  | String  | Name of table to perform rollback on |
   ```
   Maybe this to keep the word "rollback"?

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |

Review comment:
       ```suggestion
   | current_snapshot_id  | Long | The new current snapshot ID |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will no longer be able to access them.
+
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |
+| table         | ✔️  | String | Name of the new Iceberg table to create |
+| location      |     | String | Table location for new table (Defaults to catalog's default location) |

Review comment:
       ```suggestion
   | location      |     | String | Table location for the new table (defaults to the catalog's default location) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |

Review comment:
       ```suggestion
   | current_snapshot_id  | Long | The new current snapshot ID |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table

Review comment:
       ```suggestion
   **Note** this procedure invalidates all cached Spark plans that reference the affected table.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|

Review comment:
       ```suggestion
   | retain_last   |     | Int    | Length of history to preserve regardless of older_than target (defaults to 1) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |

Review comment:
       ```suggestion
   | source_snapshot_id | Long | The snapshot ID before applying the cherry pick |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1

Review comment:
       ```suggestion
   Cherry pick Snapshot 1
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.

Review comment:
       ```suggestion
   Erase all snapshots older than the current timestamp but retain the last 5 snapshots made.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |

Review comment:
       ```suggestion
   | snapshot_id   | ✔️   | Long | The snapshot ID to cherry pick |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 

Review comment:
       ```suggestion
   Hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |

Review comment:
       ```suggestion
   | older_than    | ️   | Timestamp   | Remove snapshots older than this date (defaults to 5 days) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |

Review comment:
       ```suggestion
   | deleted_manifest_lists_count | Long | Number of manifest List files deleted by this operation |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|

Review comment:
       ```suggestion
   | current_snapshot_id | Long | The current snapshot ID now that the cherry pick has been applied|
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all

Review comment:
       I don't understand what "making" means here?

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |

Review comment:
       ```suggestion
   | deleted_manifest_files_count | Long | Number of manifest files deleted by this operation |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|

Review comment:
       ```suggestion
   | migrated_files_count | Long | Number of files migrated to Iceberg |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |

Review comment:
       ```suggestion
   | deleted_data_files_count | Long | Number of data files deleted by this operation |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.

Review comment:
       ```suggestion
   the `expire_snapshots` procedure will never remove files which are still required by a non-expired snapshot.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'

Review comment:
       ```suggestion
   Migrate the table `db.sample` to an Iceberg table and add a property 'foo' set to 'bar':
   ```
   

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer

Review comment:
       ```suggestion
   Used for removing files which are no longer
   ```
   (I think the second sentence pretty much says the same thing)

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |

Review comment:
       ```suggestion
   | use_caching   | ️   | Boolean | Use Spark caching during operation (defaults to true) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure

Review comment:
       ```suggestion
   ### Expire Snapshots Procedure
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.

Review comment:
       ```suggestion
   required by an Iceberg table as they are no longer referenced in any metadata files and can thus be considered "orphaned".
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |

Review comment:
       ```suggestion
   | previous_snapshot_id | Long | The snapshot ID before the rollback |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |

Review comment:
       ```suggestion
   | older_than    | ️   | Timestamp | Remove orphan files older than this time (defaults to 3 days) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |

Review comment:
       ```suggestion
   | orphan_file_location | String | The path to each file determined to be an orphan by this command |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see

Review comment:
       ```suggestion
   A procedure that rolls back a table to a specific snapshot ID. For rollbacks based on time see
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.

Review comment:
       ```suggestion
   List all the files that are candidates for removal by performing a dry run of the `remove_orphan_files` command on this table without actually removing them.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |

Review comment:
       ```suggestion
   | location      |     | String    | Directory to look for files in (defaults to the table's location) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.

Review comment:
       ```suggestion
   Rewrites manifests in a table and co-locates metadata for partitions.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).

Review comment:
       ```suggestion
   [rollback_to_timestamp](#rollback-to-timestamp-procedure).
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |

Review comment:
       ```suggestion
   | snapshot_id   | ✔️  | Long     | The snapshot ID to rollback to |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |

Review comment:
       ```suggestion
   | dry_run       |     | Boolean   | When true, don't actually remove files (defaults to false) |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |

Review comment:
       ```suggestion
   | table         | ✔️  | String    | Name of table to perform rollback on |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |

Review comment:
       ```suggestion
   | previous_snapshot_id | Long | The snapshot ID before the rollback |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.

Review comment:
       ```suggestion
   A procedure that rolls back a table to a certain point in time.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |

Review comment:
       ```suggestion
   | current_snapshot_id  | Long | The new current snapshot ID |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )

Review comment:
       ```suggestion
       CALL catalog_name hive_prod.system.cherry_pick_snapshot(snapshot_id => 1, table => 'my_table' )
   ```
   I guess it's too late to rename the actual procedure right? NABD, I just know for example in Git it's called `cherry-pick` not `cherrypick` as it's two words, not one.

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |

Review comment:
       ```suggestion
   | snapshot_id   | ✔️  | Long     | The snapshot ID to use as current |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will no longer be able to access them.
+
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| source_table  | ✔️  | String | Source table for making the Iceberg table |

Review comment:
       ```suggestion
   | source_table  | ✔️  | String | Source Hive table for the Iceberg table |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can

Review comment:
       ```suggestion
   Creates an Iceberg version of a given table without adjusting the underlying table. The newly created table can
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 

Review comment:
       ```suggestion
   are still allowed. In addition, any operations which affect the original data files will disrupt the Snapshot's 
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |

Review comment:
       ```suggestion
   | previous_snapshot_id | Long | The snapshot ID before this change |
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the

Review comment:
       ```suggestion
   integrity. DELETE statements executed against the original Hive table will remove original data files and the
   ```
   (to be consistent with the upper-casing on line 1096, alternatively remove the upper-case from there and keep this as-is.)

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rewrite manifests for |
+| use_caching   | ️   | Boolean | Use Spark caching during operation (Defaults to true) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| rewritten_manifests_count | Int | Number of manifests which were re-written by this command |
+| added_mainfests_count     | Int | Number of new manifest files which were written by this command |
+
+#### Examples
+
+Rewrite the manifests in table `db.sample` and align manifest files with table partitioning.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample')
+```
+
+Rewrite the manifests in table `db.sample` and disable the use of Spark caching. This could be done to
+avoid memory issues on executors.
+```sql
+   CALL hive_prod.system.rewrite_manifests('db.sample', false)
+```
+
+### Rollback to Snapshot Procedure
+
+A procedure that rollbacks a table to a specific snapshot id. For rollbacks based on time see
+[rollback_to_timestamp](#rollback-to-snapshot-procedure).
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String  | Name of table to rollback |
+| snapshot_id   | ✔️  | Long     | The snapshot to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to snapshot `1`
+```sql
+   CALL hive_prod.system.rollback_to_snapshot('db.sample', 1)
+```
+
+### Rollback to Timestamp Procedure
+
+A procedure that rollbacks a table to a certain point in time.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to rollback |
+| timestamp     | ✔️  | Timestamp | The time to rollback to |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before rollback |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Rollback `db.sample` to a day ago
+```sql
+    CALL hive_prod.system.rollback_to_timestamp('db.sample', date_sub(current_date(), 1))
+```
+
+### Set Current Snapshot Procedure
+
+Sets the current snapshot for a table to a different snapshot.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String   | Name of table to change |
+| snapshot_id   | ✔️  | Long     | The snapshot to use as current |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| previous_snapshot_id | Long | The current snapshot before this procedure |
+| current_snapshot_id  | Long | The new current snapshot id |
+
+#### Example
+
+Set the current snapshot for `db.sample` to 1
+```sql
+    CALL hive_prod.system.set_current_snapshot('db.sample', 1)
+```
+
+### Snapshot Table Procedure
+
+Creates an Iceberg version of a given table without adjusting the underlying table. The new created table can
+be adjusted or written to without adjusting the underlying table. This procedure could be used to Snapshot a Hive
+production table and produce an Iceberg table referencing the same files. This new table can be used for testing before 
+a full migration to Iceberg, or just to see if certain queries would benefit from Iceberg's optimizations. When 
+INSERTS/DELETES and other statements are run on the snapshot, they will create new files in the Snapshot's location
+rather than the original table location.
+
+**Note** Because tables created by `snapshot` 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 effect the original data files will disrupt the Snapshot's 
+integrity. Delete statements executed against the original Hive table will remove original data files and the
+`snapshot` table will no longer be able to access them.
+
+[migrate](#migrate-table-procedure) which can be used without disrupting users of the original table.

Review comment:
       ```suggestion
   See also [migrate table](#migrate-table-procedure) which can be used without disrupting users of the original table.
   ```

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| deleted_data_files_count | Long | Data files deleted by this operation |
+| deleted_manifest_files_count | Long | Manifest files deleted by this operation |
+| deleted_manifest_lists_count | Long | Manifest List files deleted by this operation |
+
+#### Examples
+
+Remove snapshots older than 10 days ago, but retain the last 100 Snapshots.
+```sql
+    CALL hive_prod.system.expire_snapshots('db.sample', date_sub(current_date(), 10), 100)
+```
+
+Erase all snapshots older than the current timestamp but retains the last 5 snapshots made.
+```sql
+    CALL hive_prod.system.expire_snapshots(table => 'db.sample', retain_last => 5)
+```
+
+### Migrate Table Procedure
+
+Converts a table known to Spark in this catalog into a Iceberg table. This is manily for taking
+hive tables and fully converting them into Iceberg tables. The [snapshot](#snapshot-table-procedure) procedure 
+can be used for making  Iceberg tables without effecting the underlying table. The migrated table will preserve all
+properties set on the original table.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Table to be migrated to Iceberg |
+| properties    | ️   | Map<String, String> | Table properties to add to the table |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| migrated_files_count | Long | Number of files migrated to Iceberg|
+
+#### Examples
+
+Migrate the table db.sample into an Iceberg table and add a property 'foo' set to 'bar'
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample', map('foo', 'bar'))
+```
+
+Migrate the table db.sample
+```sql
+    CALL catalog_name hive_prod.system.migrate('db.sample')
+```
+
+### Remove Orphan Files Procedure
+
+Remove files in a directory which a given Iceberg table does not own. Used for removing files which are no longer
+required by an Iceberg table and Iceberg has no references to.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String    | Name of table to remove files from |
+| older_than    | ️   | Timestamp | Remove orphan files older than this time (Defaults to 3 days ago) |
+| location      |     | String    | Directory to look for files in (Defaults to the table's location) |
+| dry_run       |     | Boolean   | When true, don't actually remove files (Defaults to false) |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| orphan_file_location | String | A file determined to be an orphan by this command |
+
+#### Examples
+
+Remove any files in the `tablelocation/data` folder which are not known to the table `db.sample` and have a timestamp
+older than the current timestamp.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', older_than => now(), location => 'tablelocation/data')
+```
+
+List all files that would be removed by the default `remove_orphan_files` command on this table without removing them.
+```sql
+   CALL hive_prod.system.remove_orphan_files(table => 'db.sample', dry_run => true)
+```
+
+### Rewrite Manifests Procedure
+
+A procedure that rewrites manifests in a table and co-locates metadata for partitions.

Review comment:
       I think this could do with another line or two explaining in a bit more detail what this does, why you would want to use it, what the benefits are etc.

##########
File path: site/docs/spark.md
##########
@@ -814,3 +815,320 @@ This type conversion table describes how Iceberg types are converted to the Spar
 | struct                     | struct                  |               |
 | list                       | array                   |               |
 | map                        | map                     |               |
+
+## Procedures
+
+In Spark 3 Iceberg provides an SQL API for accomplishing the [maintenance actions](maintenance.md). Support for
+stored procedures is tied to the DataSourceV2 catalog and requires that the Iceberg Extensions are enabled for the
+Spark Session.
+
+### General Usage
+
+To call an Iceberg stored procedure, execute a `CALL` command against the iceberg catalog. All procedures are added to
+the `system` keyspace. Procedures can take positional or named arguments.
+
+#### Generic Call with Positional Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_1, arg_2, ... arg_n)
+```
+
+#### Generic Call with Named Arguments
+```sql
+    CALL catalog_name.system.procedure_name(arg_name_2 => arg_2, arg_name_1 => arg_1)
+```
+
+### Cherrypick Snapshot Procedure
+
+A procedure that applies changes in a given snapshot and creates a new snapshot which will
+be set as the current snapshot in a table.
+
+**Note** this procedure invalidates all cached Spark plans that reference the affected table
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to perform cherrypick on |
+| snapshot_id   | ✔️   | Long | The snapshot ID to cherrypick |
+
+#### Output
+
+| Output Name | Type | Description |
+| ------------|------|-------------|
+| source_snapshot_id | Long | The snapshot before applying the cherrypick |
+| current_snapshot_id | Long | The current snapshot now that the cherrypick has been applied|
+
+#### Examples
+
+Cherrypick Snapshot 1
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot('my_table', 1)
+```
+
+Cherrypick Snapshot 1 with named args
+```sql
+    CALL catalog_name hive_prod.system.cherrypick_snapshot(snapshot_id => 1, table => 'my_table' )
+```
+
+### Expire Snapshot Procedure
+
+Each write/update/delete/upsert/compaction in Iceberg produces a new snapshot while keeping the old data and metadata
+around for snapshot isolation and time travel. The `expire_snapshots` procedure can be used to remove older snapshots
+and their files which are no longer needed.
+
+This procedure will remove old snapshots and data files which are uniquely required by those old snapshots. This means
+the ``expire_snapshots`` procedure will never remove files which are still required by a non-expired snapshot.
+
+#### Usage
+
+| Argument Name | Required? | Type | Description |
+|---------------|-----------|------|-------------|
+| table         | ✔️  | String | Name of table to expire snapshots from |
+| older_than    | ️   | Timestamp   | Remove snapshots older than this date (Defaults to 5 days ago) |
+| retain_last   |     | Int    | Length of history to preserve regardless of older_than target (Defaults to 1)|

Review comment:
       By "length of history" do you mean "number of snapshots"?




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

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



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