Skip to main content
Version: 0.14.1

Spark Guide

This guide provides a quick peek at Hudi's capabilities using Spark. Using Spark Datasource APIs(both scala and python) and using Spark SQL, we will walk through code snippets that allows you to insert, update, delete and query a Hudi table.

Setup

Hudi works with Spark-2.4.3+ & Spark 3.x versions. You can follow instructions here for setting up Spark.

Spark 3 Support Matrix

HudiSupported Spark 3 version
0.14.x3.4.x (default build), 3.3.x, 3.2.x, 3.1.x, 3.0.x
0.13.x3.3.x (default build), 3.2.x, 3.1.x
0.12.x3.3.x (default build), 3.2.x, 3.1.x
0.11.x3.2.x (default build, Spark bundle only), 3.1.x
0.10.x3.1.x (default build), 3.0.x
0.7.0 - 0.9.03.0.x
0.6.0 and priornot supported

The default build Spark version indicates how we build hudi-spark3-bundle.

Change summary

In 0.14.0, we introduced the support for Spark 3.4.x and bring back the support for Spark 3.0.x. In 0.12.0, we introduced the experimental support for Spark 3.3.0. In 0.11.0, there are changes on using Spark bundles, please refer to 0.11.0 release notes for detailed instructions.

Spark Shell/SQL

From the extracted directory run spark-shell with Hudi:

# For Spark versions: 3.2 - 3.4
export SPARK_VERSION=3.4
spark-shell --packages org.apache.hudi:hudi-spark$SPARK_VERSION-bundle_2.12:0.14.1 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog' --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar'
# For Spark versions: 3.0 - 3.1
export SPARK_VERSION=3.1
spark-shell --packages org.apache.hudi:hudi-spark$SPARK_VERSION-bundle_2.12:0.14.1 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar'
# For Spark version: 2.4
spark-shell --packages org.apache.hudi:hudi-spark2.4-bundle_2.11:0.14.1 --conf 'spark.serializer=org.apache.spark.serializer.KryoSerializer' --conf 'spark.sql.extensions=org.apache.spark.sql.hudi.HoodieSparkSessionExtension' --conf 'spark.kryo.registrator=org.apache.spark.HoodieSparkKryoRegistrar'
for Spark 3.2 and higher versions

Use scala 2.12 builds with an additional config: --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.hudi.catalog.HoodieCatalog'

Setup project

Below, we do imports and setup the table name and corresponding base path.

// spark-shell
import scala.collection.JavaConversions._
import org.apache.spark.sql.SaveMode._
import org.apache.hudi.DataSourceReadOptions._
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.table.HoodieTableConfig._
import org.apache.hudi.config.HoodieWriteConfig._
import org.apache.hudi.keygen.constant.KeyGeneratorOptions._
import org.apache.hudi.common.model.HoodieRecord
import spark.implicits._

val tableName = "trips_table"
val basePath = "file:///tmp/trips_table"

Create Table

First, let's create a Hudi table. Here, we use a partitioned table for illustration, but Hudi also supports non-partitioned tables.

// scala
// First commit will auto-initialize the table, if it did not exist in the specified base path.

Insert data

Generate some new records as a DataFrame and write the DataFrame into a Hudi table. Since, this is the first write, it will also auto-create the table.

// spark-shell
val columns = Seq("ts","uuid","rider","driver","fare","city")
val data =
Seq((1695159649087L,"334e26e9-8355-45cc-97c6-c31daf0df330","rider-A","driver-K",19.10,"san_francisco"),
(1695091554788L,"e96c4396-3fad-413a-a942-4cb36106d721","rider-C","driver-M",27.70 ,"san_francisco"),
(1695046462179L,"9909a8b1-2d15-4d3d-8ec9-efc48c536a00","rider-D","driver-L",33.90 ,"san_francisco"),
(1695516137016L,"e3cf430c-889d-4015-bc98-59bdce1e530c","rider-F","driver-P",34.15,"sao_paulo" ),
(1695115999911L,"c8abbe79-8d89-47ea-b4ce-4d224bae5bfa","rider-J","driver-T",17.85,"chennai"));

var inserts = spark.createDataFrame(data).toDF(columns:_*)
inserts.write.format("hudi").
option(PARTITIONPATH_FIELD_NAME.key(), "city").
option(TABLE_NAME, tableName).
mode(Overwrite).
save(basePath)
Mapping to Hudi write operations

Hudi provides a wide range of write operations - both batch and incremental - to write data into Hudi tables, with different semantics and performance. When record keys are not configured (see keys below), bulk_insert will be chosen as the write operation, matching the out-of-behavior of Spark's Parquet Datasource.

Query data

Hudi tables can be queried back into a DataFrame or Spark SQL.

// spark-shell
val tripsDF = spark.read.format("hudi").load(basePath)
tripsDF.createOrReplaceTempView("trips_table")

spark.sql("SELECT uuid, fare, ts, rider, driver, city FROM trips_table WHERE fare > 20.0").show()
spark.sql("SELECT _hoodie_commit_time, _hoodie_record_key, _hoodie_partition_path, rider, driver, fare FROM trips_table").show()

Update data

Hudi tables can be updated by streaming in a DataFrame or using a standard UPDATE statement.

// Lets read data from target Hudi table, modify fare column for rider-D and update it. 
val updatesDf = spark.read.format("hudi").load(basePath).filter($"rider" === "rider-D").withColumn("fare", col("fare") * 10)

updatesDf.write.format("hudi").
option(OPERATION_OPT_KEY, "upsert").
option(PARTITIONPATH_FIELD_NAME.key(), "city").
option(TABLE_NAME, tableName).
mode(Append).
save(basePath)
Key requirements

Updates with spark-datasource is feasible only when the source dataframe contains Hudi's meta fields or a key field is configured. Notice that the save mode is now Append. In general, always use append mode unless you are trying to create the table for the first time.

Querying the data again will now show updated records. Each write operation generates a new commit. Look for changes in _hoodie_commit_time, fare fields for the given _hoodie_record_key value from a previous commit.

Merging Data

// spark-shell
val adjustedFareDF = spark.read.format("hudi").
load(basePath).limit(2).
withColumn("fare", col("fare") * 10)
adjustedFareDF.write.format("hudi").
option("hoodie.datasource.write.payload.class","com.payloads.CustomMergeIntoConnector").
mode(Append).
save(basePath)
// Notice Fare column has been updated but all other columns remain intact.
spark.read.format("hudi").load(basePath).show()

The com.payloads.CustomMergeIntoConnector adds adjusted fare values to the original table and preserves all other fields. Refer here for sample implementation of com.payloads.CustomMergeIntoConnector.

Delete data

Delete operation removes the records specified from the table. For example, this code snippet deletes records for the HoodieKeys passed in. Check out the deletion section for more details.

// spark-shell
// Lets delete rider: rider-D
val deletesDF = spark.read.format("hudi").load(basePath).filter($"rider" === "rider-F")

deletesDF.write.format("hudi").
option(OPERATION_OPT_KEY, "delete").
option(PARTITIONPATH_FIELD_NAME.key(), "city").
option(TABLE_NAME, tableName).
mode(Append).
save(basePath)

Querying the data again will not show the deleted record.

Key requirements

Deletes with spark-datasource is supported only when the source dataframe contains Hudi's meta fields or a key field is configured. Notice that the save mode is again Append.

Time Travel Query

Hudi supports time travel query to query the table as of a point-in-time in history. Three timestamp formats are supported as illustrated below.

spark.read.format("hudi").
option("as.of.instant", "20210728141108100").
load(basePath)

spark.read.format("hudi").
option("as.of.instant", "2021-07-28 14:11:08.200").
load(basePath)

// It is equal to "as.of.instant = 2021-07-28 00:00:00"
spark.read.format("hudi").
option("as.of.instant", "2021-07-28").
load(basePath)

Incremental query

Hudi provides the unique capability to obtain a set of records that changed between a start and end commit time, providing you with the "latest state" for each such record as of the end commit time. By default, Hudi tables are configured to support incremental queries, using record level metadata tracking.

Below, we fetch changes since a given begin time while the end time defaults to the latest commit on the table. Users can also specify an end time using END_INSTANTTIME.key() option.

// spark-shell
spark.read.format("hudi").load(basePath).createOrReplaceTempView("trips_table")

val commits = spark.sql("SELECT DISTINCT(_hoodie_commit_time) AS commitTime FROM trips_table ORDER BY commitTime").map(k => k.getString(0)).take(50)
val beginTime = commits(commits.length - 2) // commit time we are interested in

// incrementally query data
val tripsIncrementalDF = spark.read.format("hudi").
option(QUERY_TYPE.key(), QUERY_TYPE_INCREMENTAL_OPT_VAL).
option(BEGIN_INSTANTTIME.key(), 0).
load(basePath)
tripsIncrementalDF.createOrReplaceTempView("trips_incremental")

spark.sql("SELECT `_hoodie_commit_time`, fare, rider, driver, uuid, ts FROM trips_incremental WHERE fare > 20.0").show()

Change Data Capture Query

Hudi also exposes first-class support for Change Data Capture (CDC) queries. CDC queries are useful for applications that need to obtain all the changes, along with before/after images of records, given a commit time range.

// spark-shell
// Lets first insert data to a new table with cdc enabled.
val columns = Seq("ts","uuid","rider","driver","fare","city")
val data =
Seq((1695158649187L,"334e26e9-8355-45cc-97c6-c31daf0df330","rider-A","driver-K",19.10,"san_francisco"),
(1695091544288L,"e96c4396-3fad-413a-a942-4cb36106d721","rider-B","driver-L",27.70 ,"san_paulo"),
(1695046452379L,"9909a8b1-2d15-4d3d-8ec9-efc48c536a00","rider-C","driver-M",33.90 ,"san_francisco"),
(1695332056404L,"1dced545-862b-4ceb-8b43-d2a568f6616b","rider-D","driver-N",93.50,"chennai"));
var df = spark.createDataFrame(data).toDF(columns:_*)

// Insert data
df.write.format("hudi").
option(PARTITIONPATH_FIELD_NAME.key(), "city").
option(CDC_ENABLED.key(), "true").
option(TABLE_NAME, tableName).
mode(Overwrite).
save(basePath)

// Update fare for riders: rider-A and rider-B
val updatesDf = spark.read.format("hudi").load(basePath).filter($"rider" === "rider-A" || $"rider" === "rider-B").withColumn("fare", col("fare") * 10)

updatesDf.write.format("hudi").
option(OPERATION_OPT_KEY, "upsert").
option(PARTITIONPATH_FIELD_NAME.key(), "city").
option(CDC_ENABLED.key(), "true").
option(TABLE_NAME, tableName).
mode(Append).
save(basePath)


// Query CDC data
spark.read.option(BEGIN_INSTANTTIME.key(), 0).
option(QUERY_TYPE.key(), QUERY_TYPE_INCREMENTAL_OPT_VAL).
option(INCREMENTAL_FORMAT.key(), "cdc").
format("hudi").load(basePath).show(false)
Key requirements

Note that CDC queries are currently only supported on Copy-on-Write tables.

Table Types

The examples thus far have showcased one of the two table types, that Hudi supports - Copy-on-Write (COW) tables. Hudi also supports a more advanced write-optimized table type called Merge-on-Read (MOR) tables, that can balance read and write performance in a more flexible manner. See table types for more details.

Any of these examples can be run on a Merge-on-Read table by simply changing the table type to MOR, while creating the table, as below.

// spark-shell
inserts.write.format("hudi").
...
option(TABLE_TYPE.key(), "MERGE_ON_READ").
...

Keys

Hudi also allows users to specify a record key, which will be used to uniquely identify a record within a Hudi table. This is useful and critical to support features like indexing and clustering, which speed up upserts and queries respectively, in a consistent manner. Some of the other benefits of keys are explained in detail here. To this end, Hudi supports a wide range of built-in key generators, that make it easy to generate record keys for a given table. In the absence of a user configured key, Hudi will auto generate record keys, which are highly compressible.

// spark-shell
inserts.write.format("hudi").
...
option(RECORDKEY_FIELD.key(), "uuid").
...
Implications of defining record keys

Configuring keys for a Hudi table, has a new implications on the table. If record key is set by the user, upsert is chosen as the write operation. Also if a record key is configured, then it's also advisable to specify a precombine or ordering field, to correctly handle cases where the source data has multiple records with the same key. See section below.

Ordering Field

Hudi also allows users to specify a precombine field, which will be used to order and resolve conflicts between multiple versions of the same record. This is very important for use-cases like applying database CDC logs to a Hudi table, where a given record may be appear multiple times in the source data due to repeated upstream updates. Hudi also uses this mechanism to support out-of-order data arrival into a table, where records may need to be resolved in a different order than their commit time. For e.g using a created_at timestamp field as the precombine field will prevent older versions of a record from overwriting newer ones or being exposed to queries, even if they are written at a later commit time to the table. This is one of the key features, that makes Hudi, best suited for dealing with streaming data.

// spark-shell 
updatesDf.write.format("hudi").
...
option(PRECOMBINE_FIELD_NAME.key(), "ts").
...

Where to go from here?

You can also build hudi yourself and try this quickstart using --jars <path to spark bundle jar>(see also build with scala 2.12) for more info. If you are looking for ways to migrate your existing data to Hudi, refer to migration guide.

Spark SQL Reference

For advanced usage of spark SQL, please refer to Spark SQL DDL and Spark SQL DML reference guides. For alter table commands, check out this. Stored procedures provide a lot of powerful capabilities using Hudi SparkSQL to assist with monitoring, managing and operating Hudi tables, please check this out.

Streaming workloads

Hudi provides industry-leading performance and functionality for streaming data.

Hudi Streamer - Hudi provides an incremental ingestion/ETL tool - HoodieStreamer, to assist with ingesting data into Hudi from various different sources in a streaming manner, with powerful built-in capabilities like auto checkpointing, schema enforcement via schema provider, transformation support, automatic table services and so on.

Structured Streaming - Hudi supports Spark Structured Streaming reads and writes as well. Please see here for more.

Check out more information on modeling data in Hudi and different ways to writing Hudi Tables.

Dockerized Demo

Even as we showcased the core capabilities, Hudi supports a lot more advanced functionality that can make it easy to get your transactional data lakes up and running quickly, across a variety query engines like Hive, Flink, Spark, Presto, Trino and much more. We have put together a demo video that showcases all of this on a docker based setup with all dependent systems running locally. We recommend you replicate the same setup and run the demo yourself, by following steps here to get a taste for it.