Using XTable to translate from Iceberg to Hudi & Delta Lake with a File System Catalog like S3
While going through some of the recent issues in the Apache XTable repository, I stumbled upon this error from a user who was attempting to translate their Apache Iceberg table into Delta Lake.
TL;DR — The issue at hand is not directly tied to the translation process itself, but primarily relates to a configuration aspect within Iceberg tables.
Let’s first understand what XTable is flagging as problematic.
2024-04-30 10:24:25 WARN org.apache.iceberg.hadoop.HadoopTableOperations:325 - Error reading version hint file s3:///iceberg_table_1/metadata/version-hint.text
java.io.FileNotFoundException: No such file or directory: s3:////iceberg_table_1/metadata/version-hint.text
The error message is pretty straightforward —it is a FileNotFoundException
. Specifically it states that the version-hint.text
file is not found in the metadata
directory of the Iceberg table.
Background
So, what exactly is the version-hint.text
file? A critical aspect to note when working with Iceberg table is that configuring a catalog is one of the initial steps required.
A catalog in Iceberg acts as a logical namespace that stores metadata about tables and offers a consistent, unified view of the data to different compute engines. Most importantly, a catalog provides consistency guarantees in scenarios involving concurrent writing. In a broader sense, the catalog maintains a reference to the latest metadata file, and each time a writer commits changes, this reference is atomically updated.
Technically, you can use any database or file system as a catalog. The version-hint.text
file comes into the picture when you choose a file system catalog, such as HDFS or Amazon S3. This is usually referred to as Hadoop catalog in the Iceberg context.
Upon inspecting the content of this file, you will find a ‘number’, which indicates the version of the metadata file.
Error Troubleshooting
So, coming back to the error, XTable attempts to use the source table metadata (in this case, Iceberg) to facilitate translation into Delta Lake. Since the version-hint.text
file is the initial reference point for accessing the latest metadata file, and because this file is missing from the metadata folder, XTable encounters an error.
As a first step, I looked into the configuration used by the user to create the source Iceberg table.
pyspark
--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"
We can see that the user configured the catalog type as hive
. However, the necessary Hive-related configurations, such as the Thrift URL, were not provided, which is the root of the problem. To summarize quickly, if you plan to use Iceberg with a specific catalog, the correct configurations must be in place to generate the appropriate catalog file.
Solution
The solution to this problem is simply to use the correct configuration for the Iceberg source table and this is what I recommended.
spark.sql.catalog.hadoop_prod = org.apache.iceberg.spark.SparkCatalog
spark.sql.catalog.hadoop_prod.type = hadoop
spark.sql.catalog.hadoop_prod.warehouse = s3a://your-bucket
While this issue primarily concerns how to set up and configure Iceberg tables, it also reminded me that XTable’s official documentation lacks an example of how to use an Iceberg table as a source with a catalog.
So, I decided to show an end-to-end example to create and configure an Iceberg table with a file system catalog like S3 and translate it into Hudi and Delta Lake using XTable.
Create Iceberg Table with S3 Catalog
import pyspark
from pyspark.sql import SparkSession
import os
conf = (
pyspark.SparkConf()
.setAppName('app_name')
.set('spark.jars.packages', 'org.apache.hadoop:hadoop-aws:3.3.4,org.apache.iceberg:iceberg-spark-runtime-3.3_2.12:1.4.3,software.amazon.awssdk:bundle:2.17.178,software.amazon.awssdk:url-connection-client:2.17.178')
.set('spark.sql.extensions', 'org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions')
.set('spark.sql.catalog.hdfs_catalog', 'org.apache.iceberg.spark.SparkCatalog')
.set('spark.sql.catalog.hdfs_catalog.type', 'hadoop')
.set('spark.sql.catalog.hdfs_catalog.warehouse', 's3a://my-bucket/new_iceberg/')
.set('spark.sql.catalog.hdfs_catalog.io-impl', 'org.apache.iceberg.aws.s3.S3FileIO')
)
spark = SparkSession.builder.config(conf=conf).getOrCreate()
print("Spark Running")
spark.sql("CREATE TABLE hdfs_catalog.table1 (name string) USING iceberg")
spark.sql("INSERT INTO hdfs_catalog.table1 VALUES ('Harry'), ('Dipankar'), ('Mary')")
If we take a look at the S3 bucket (data lake storage), we can now see the version-hint.text
file generated.
Now, let’s create the my_config.yaml
file where we will specify the source and target formats along with the location of the source table.
sourceFormat: ICEBERG
targetFormats:
- DELTA
- HUDI
datasets:
-
tableBasePath: s3://my-bucket/new_iceberg/table1/
tableDataPath: s3://my-bucket/new_iceberg/table1/data
tableName: table1
Finally, we will run the sync job for XTable to start translating.
java -jar utilities/target/utilities-0.1.0-SNAPSHOT-bundled.jar --datasetConfig my_config.yaml
Now if we check the S3 bucket, we can see the expected Hudi and Delta table metadata.
We are now ready to consume these tables with the compute engine of our choice, without being restricted to just Iceberg. This capability underlines the interoperability of various lakehouse table formats.
I hope this brief blog has provided insight into how to use Apache XTable to translate Iceberg tables with a file system-based catalog (like S3) into other formats such as Hudi and Delta Lake.