Disaster Recovery is very much mission-critical for any software. Especially when it comes to data systems, the impact could be very serious leading to delay in business decisions or even wrong business decisions at times. Apache Hudi has two operations to assist you in recovering data from a previous state: savepoint and restore.

Savepoint

As the name suggest, savepoint saves the table as of the commit time, so that it lets you restore the table to this savepoint at a later point in time if need be. Care is taken to ensure cleaner will not clean up any files that are savepointed. On similar lines, savepoint cannot be triggered on a commit that is already cleaned up. In simpler terms, this is synonymous to taking a backup, just that we don’t make a new copy of the table, but just save the state of the table elegantly so that we can restore it later when in need.

Restore

This operation lets you restore your table to one of the savepoint commit. This operation cannot be undone (or reversed) and so care should be taken before doing a restore. Hudi will delete all data files and commit files (timeline files) greater than the savepoint commit to which the table is being restored. You should pause all writes to the table when performing a restore since they are likely to fail while the restore is in progress. Also, reads could also fail since snapshot queries will be hitting latest files which has high possibility of getting deleted with restore.

Runbook

Savepoint and restore can only be triggered from hudi-cli. Let’s walk through an example of how one can take savepoint and later restore the state of the table.

Let’s create a hudi table via spark-shell and trigger a batch of inserts.

  1. import org.apache.hudi.QuickstartUtils._
  2. import scala.collection.JavaConversions._
  3. import org.apache.spark.sql.SaveMode._
  4. import org.apache.hudi.DataSourceReadOptions._
  5. import org.apache.hudi.DataSourceWriteOptions._
  6. import org.apache.hudi.config.HoodieWriteConfig._
  7. val tableName = "hudi_trips_cow"
  8. val basePath = "file:///tmp/hudi_trips_cow"
  9. val dataGen = new DataGenerator
  10. val inserts = convertToStringList(dataGen.generateInserts(10))
  11. val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
  12. df.write.format("hudi").
  13. options(getQuickstartWriteConfigs).
  14. option("hoodie.datasource.write.precombine.field", "ts").
  15. option("hoodie.datasource.write.recordkey.field", "uuid").
  16. option("hoodie.datasource.write.partitionpath.field", "partitionpath").
  17. option("hoodie.table.name", tableName).
  18. mode(Overwrite).
  19. save(basePath)

Let’s add four more batches of inserts.

  1. for (_ <- 1 to 4) {
  2. val inserts = convertToStringList(dataGen.generateInserts(10))
  3. val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
  4. df.write.format("hudi").
  5. options(getQuickstartWriteConfigs).
  6. option("hoodie.datasource.write.precombine.field", "ts").
  7. option("hoodie.datasource.write.recordkey.field", "uuid").
  8. option("hoodie.datasource.write.partitionpath.field", "partitionpath").
  9. option("hoodie.table.name", tableName).
  10. mode(Append).
  11. save(basePath)
  12. }

Total record count should be 50.

  1. val tripsSnapshotDF = spark.
  2. read.
  3. format("hudi").
  4. load(basePath)
  5. tripsSnapshotDF.createOrReplaceTempView("hudi_trips_snapshot")
  6. spark.sql("select count(partitionpath, uuid) from hudi_trips_snapshot").show()
  7. +--------------------------+
  8. |count(partitionpath, uuid)|
  9. +--------------------------+
  10. | 50|
  11. +--------------------------+

Let’s take a look at the timeline after 5 batches of inserts.

  1. ls -ltr /tmp/hudi_trips_cow/.hoodie
  2. total 128
  3. drwxr-xr-x 2 nsb wheel 64 Jan 28 16:00 archived
  4. -rw-r--r-- 1 nsb wheel 546 Jan 28 16:00 hoodie.properties
  5. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:00 20220128160040171.commit.requested
  6. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:00 20220128160040171.inflight
  7. -rw-r--r-- 1 nsb wheel 4374 Jan 28 16:00 20220128160040171.commit
  8. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:01 20220128160124637.commit.requested
  9. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:01 20220128160124637.inflight
  10. -rw-r--r-- 1 nsb wheel 4414 Jan 28 16:01 20220128160124637.commit
  11. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160226172.commit.requested
  12. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160226172.inflight
  13. -rw-r--r-- 1 nsb wheel 4427 Jan 28 16:02 20220128160226172.commit
  14. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160229636.commit.requested
  15. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160229636.inflight
  16. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160229636.commit
  17. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160245447.commit.requested
  18. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160245447.inflight
  19. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160245447.commit

Let’s trigger a savepoint as of the latest commit. Savepoint can only be done via hudi-cli.

  1. ./hudi-cli.sh
  2. connect --path /tmp/hudi_trips_cow/
  3. commits show
  4. set --conf SPARK_HOME=<SPARK_HOME>
  5. savepoint create --commit 20220128160245447 --sparkMaster local[2]

Make sure you replace 20220128160245447 with the latest commit in your table.

Let’s check the timeline after savepoint.

  1. ls -ltr /tmp/hudi_trips_cow/.hoodie
  2. total 136
  3. drwxr-xr-x 2 nsb wheel 64 Jan 28 16:00 archived
  4. -rw-r--r-- 1 nsb wheel 546 Jan 28 16:00 hoodie.properties
  5. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:00 20220128160040171.commit.requested
  6. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:00 20220128160040171.inflight
  7. -rw-r--r-- 1 nsb wheel 4374 Jan 28 16:00 20220128160040171.commit
  8. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:01 20220128160124637.commit.requested
  9. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:01 20220128160124637.inflight
  10. -rw-r--r-- 1 nsb wheel 4414 Jan 28 16:01 20220128160124637.commit
  11. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160226172.commit.requested
  12. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160226172.inflight
  13. -rw-r--r-- 1 nsb wheel 4427 Jan 28 16:02 20220128160226172.commit
  14. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160229636.commit.requested
  15. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160229636.inflight
  16. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160229636.commit
  17. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160245447.commit.requested
  18. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160245447.inflight
  19. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160245447.commit
  20. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:05 20220128160245447.savepoint.inflight
  21. -rw-r--r-- 1 nsb wheel 1168 Jan 28 16:05 20220128160245447.savepoint

You could notice that savepoint meta files are added which keeps track of the files that are part of the latest table snapshot.

Now, let’s continue adding three more batches of inserts.

  1. for (_ <- 1 to 3) {
  2. val inserts = convertToStringList(dataGen.generateInserts(10))
  3. val df = spark.read.json(spark.sparkContext.parallelize(inserts, 2))
  4. df.write.format("hudi").
  5. options(getQuickstartWriteConfigs).
  6. option("hoodie.datasource.write.precombine.field", "ts").
  7. option("hoodie.datasource.write.recordkey.field", "uuid").
  8. option("hoodie.datasource.write.partitionpath.field", "partitionpath").
  9. option("hoodie.table.name", tableName).
  10. mode(Append).
  11. save(basePath)
  12. }

Total record count will be 80 since we have done 8 batches in total. (5 until savepoint and 3 after savepoint)

  1. val tripsSnapshotDF = spark.
  2. read.
  3. format("hudi").
  4. load(basePath)
  5. tripsSnapshotDF.createOrReplaceTempView("hudi_trips_snapshot")
  6. spark.sql("select count(partitionpath, uuid) from hudi_trips_snapshot").show()
  7. +--------------------------+
  8. |count(partitionpath, uuid)|
  9. +--------------------------+
  10. | 80|
  11. +--------------------------+

Let’s say something bad happened, and you want to restore your table to an older snapshot. As we called out earlier, we can trigger restore only from hudi-cli. And do remember to bring down all of your writer processes while doing a restore.

Let’s checkout timeline once, before we trigger the restore.

  1. ls -ltr /tmp/hudi_trips_cow/.hoodie
  2. total 208
  3. drwxr-xr-x 2 nsb wheel 64 Jan 28 16:00 archived
  4. -rw-r--r-- 1 nsb wheel 546 Jan 28 16:00 hoodie.properties
  5. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:00 20220128160040171.commit.requested
  6. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:00 20220128160040171.inflight
  7. -rw-r--r-- 1 nsb wheel 4374 Jan 28 16:00 20220128160040171.commit
  8. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:01 20220128160124637.commit.requested
  9. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:01 20220128160124637.inflight
  10. -rw-r--r-- 1 nsb wheel 4414 Jan 28 16:01 20220128160124637.commit
  11. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160226172.commit.requested
  12. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160226172.inflight
  13. -rw-r--r-- 1 nsb wheel 4427 Jan 28 16:02 20220128160226172.commit
  14. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160229636.commit.requested
  15. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160229636.inflight
  16. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160229636.commit
  17. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160245447.commit.requested
  18. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160245447.inflight
  19. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160245447.commit
  20. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:05 20220128160245447.savepoint.inflight
  21. -rw-r--r-- 1 nsb wheel 1168 Jan 28 16:05 20220128160245447.savepoint
  22. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:06 20220128160620557.commit.requested
  23. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:06 20220128160620557.inflight
  24. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:06 20220128160620557.commit
  25. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:06 20220128160627501.commit.requested
  26. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:06 20220128160627501.inflight
  27. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:06 20220128160627501.commit
  28. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:06 20220128160630785.commit.requested
  29. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:06 20220128160630785.inflight
  30. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:06 20220128160630785.commit

If you are continuing in the same hudi-cli session, you can just execute refresh so that table state gets refreshed to its latest state. If not, connect to the table again.

  1. ./hudi-cli.sh
  2. connect --path /tmp/hudi_trips_cow/
  3. commits show
  4. set --conf SPARK_HOME=<SPARK_HOME>
  5. savepoints show
  6. ╔═══════════════════╗
  7. SavepointTime
  8. ╠═══════════════════╣
  9. 20220128160245447
  10. ╚═══════════════════╝
  11. savepoint rollback --savepoint 20220128160245447 --sparkMaster local[2]

Make sure you replace 20220128160245447 with the latest savepoint in your table.

Hudi table should have been restored to the savepointed commit 20220128160245447. Both data files and timeline files should have been deleted.

  1. ls -ltr /tmp/hudi_trips_cow/.hoodie
  2. total 152
  3. drwxr-xr-x 2 nsb wheel 64 Jan 28 16:00 archived
  4. -rw-r--r-- 1 nsb wheel 546 Jan 28 16:00 hoodie.properties
  5. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:00 20220128160040171.commit.requested
  6. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:00 20220128160040171.inflight
  7. -rw-r--r-- 1 nsb wheel 4374 Jan 28 16:00 20220128160040171.commit
  8. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:01 20220128160124637.commit.requested
  9. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:01 20220128160124637.inflight
  10. -rw-r--r-- 1 nsb wheel 4414 Jan 28 16:01 20220128160124637.commit
  11. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160226172.commit.requested
  12. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160226172.inflight
  13. -rw-r--r-- 1 nsb wheel 4427 Jan 28 16:02 20220128160226172.commit
  14. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160229636.commit.requested
  15. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160229636.inflight
  16. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160229636.commit
  17. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:02 20220128160245447.commit.requested
  18. -rw-r--r-- 1 nsb wheel 2594 Jan 28 16:02 20220128160245447.inflight
  19. -rw-r--r-- 1 nsb wheel 4428 Jan 28 16:02 20220128160245447.commit
  20. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:05 20220128160245447.savepoint.inflight
  21. -rw-r--r-- 1 nsb wheel 1168 Jan 28 16:05 20220128160245447.savepoint
  22. -rw-r--r-- 1 nsb wheel 0 Jan 28 16:07 20220128160732437.restore.inflight
  23. -rw-r--r-- 1 nsb wheel 4152 Jan 28 16:07 20220128160732437.restore

Let’s check the total record count in the table. Should match the records we had, just before we triggered the savepoint.

  1. val tripsSnapshotDF = spark.
  2. read.
  3. format("hudi").
  4. load(basePath)
  5. tripsSnapshotDF.createOrReplaceTempView("hudi_trips_snapshot")
  6. spark.sql("select count(partitionpath, uuid) from hudi_trips_snapshot").show()
  7. +--------------------------+
  8. |count(partitionpath, uuid)|
  9. +--------------------------+
  10. | 50|
  11. +--------------------------+

As you could see, entire table state is restored back to the commit which was savepointed. Users can choose to trigger savepoint at regular cadence and keep deleting older savepoints when new ones are created. hudi-cli has a command savepoint delete to assist in deleting a savepoint. Please do remember that cleaner may not clean the files that are savepointed. And so users should ensure they delete the savepoints from time to time. If not, the storage reclamation may not happen.

Note: Savepoint and restore for MOR table is available only from 0.11.

Videos