MIGRATING TO ICEBERG
REGISTER TABLE
is a useful utility for migrating an existing Iceberg table to another catalog. A catalog holds a reference to a specific Iceberg metadata file, so it’s important that the correct file is selected to register with the target catalog.
In this example, we’ll migrate an Iceberg table from an Apache Hive metastore to an Iceberg REST catalog. No metadata or data files are modified during this procedure, but the target catalog will now be able to reference the table.
It’s important to note that each catalog has a separate pointer to the metadata file, so changes made after registering with the new catalog will diverge. Only one catalog should be considered the source of truth. Otherwise, procedures like compaction, snapshot expiration or orphan file cleanup can delete files that the other catalog’s table continues to consider valid.
Setting up a test environment
This procedure requires connecting to a Hive Metastore. For the Spark configuration and instructions for running a metastore locally, see this chapter’s background section on Connecting to a Hive metastore.
In addition, this recipe also registers the table in a second catalog, rest_catalog
, that uses Iceberg’s REST catalog protocol. As in the background section, the source catalog is called spark_catalog
. Here is example configuration for running Spark with both catalogs:
# Start a Spark SQL shell (3.5.0)
./bin/spark-sql \
--packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.4.2 \
--conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \
--conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \
--conf spark.sql.catalog.spark_catalog.type=hive \
--conf spark.sql.catalog.spark_catalog.uri=thrift://localhost:9083 \
--conf spark.sql.catalog.spark_catalog.warehouse=/tmp/hive/data/warehouse \
--conf spark.sql.catalog.rest_catalog=org.apache.iceberg.spark.SparkCatalog \
--conf spark.sql.catalog.rest_catalog.type=rest \
--conf spark.sql.catalog.rest_catalog.uri=http://localhost:8181 \
--conf spark.sql.warehouse.dir=/tmp/hive/data/warehouse \
--conf spark.hadoop.hive.metastore.uris=thrift://localhost:9083
Creating an example table
To exercise this procedure, first create a table in the Hive catalog as the source, then add some data.
CREATE DATABASE spark_catalog.cookbook;
CREATE DATABASE rest_catalog.cookbook;
CREATE TABLE spark_catalog.cookbook.reg1 (s string) USING ICEBERG;
-- Time taken: 0.091 seconds
-- Insert a row to create a data file
INSERT INTO spark_catalog.cookbook.reg1 values ('move to rest');
-- Time taken: 0.621 seconds
Running the register_table procedure
The register_table
procedure needs two pieces of information:
- A metadata file location
- A name for the table that will be created using the metadata file
An easy way to find the latest metadata file for an existing table is to query the metadata_log_entries
metadata table.
-- Lookup the latest metadata file
SELECT file FROM spark_catalog.cookbook.reg1.metadata_log_entries;
file:/tmp/hive/data/warehouse/cookbook.db/reg1/metadata/00000-73db840f-788d-444f-a853-7de4e967c99b.metadata.json file:/tmp/hive/data/warehouse/cookbook.db/reg1/metadata/00001-e9fe8740-53c2-47f3-bda4-a59be19502fe.metadata.json
-- Time taken: 1.079 seconds, Fetched 2 row(s)
Next, use the latest metadata location to register the table metadata with the REST catalog.
-- Register the metadata file above as a new table in the target catalog
CALL rest_catalog.system.register_table (
table => 'cookbook.reg1',
metadata_file => 'file:/tmp/hive/data/warehouse/cookbook.db/reg1/metadata/00001-e9fe8740-53c2-47f3-bda4-a59be19502fe.metadata.json'
);
SHOW TABLES IN rest_catalog.cookbook;
-- reg1
SELECT * FROM rest_catalog.cookbook.reg1;
-- move to rest
Now that the registration is complete, rest_catalog.cookbook.reg1
is identical to the original Iceberg table and the data can be queried.
To avoid confusion about which table is the source or truth, remember to remove the old reference using DROP TABLE
as soon as possible.