Encryption

Since Hudi 0.11.0, Spark 3.2 support has been added and accompanying that, Parquet 1.12 has been included, which brings encryption feature to Hudi. In this section, we will show a guide on how to enable encryption in Hudi tables.

Encrypt Copy-on-Write tables

First, make sure Hudi Spark 3.2 bundle jar is used.

Then, set the following Parquet configurations to make data written to Hudi COW tables encrypted.

  1. // Activate Parquet encryption, driven by Hadoop properties
  2. jsc.hadoopConfiguration().set("parquet.crypto.factory.class", "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory")
  3. // Explicit master keys (base64 encoded) - required only for mock InMemoryKMS
  4. jsc.hadoopConfiguration().set("parquet.encryption.kms.client.class" , "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS")
  5. jsc.hadoopConfiguration().set("parquet.encryption.key.list", "k1:AAECAwQFBgcICQoLDA0ODw==, k2:AAECAAECAAECAAECAAECAA==")
  6. // Write encrypted dataframe files.
  7. // Column "rider" will be protected with master key "key2".
  8. // Parquet file footers will be protected with master key "key1"
  9. jsc.hadoopConfiguration().set("parquet.encryption.footer.key", "k1")
  10. jsc.hadoopConfiguration().set("parquet.encryption.column.keys", "k2:rider")
  11. spark.read().format("org.apache.hudi").load("path").show();

Here is an example.

  1. JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext());
  2. jsc.hadoopConfiguration().set("parquet.crypto.factory.class", "org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory");
  3. jsc.hadoopConfiguration().set("parquet.encryption.kms.client.class" , "org.apache.parquet.crypto.keytools.mocks.InMemoryKMS");
  4. jsc.hadoopConfiguration().set("parquet.encryption.footer.key", "k1");
  5. jsc.hadoopConfiguration().set("parquet.encryption.column.keys", "k2:rider");
  6. jsc.hadoopConfiguration().set("parquet.encryption.key.list", "k1:AAECAwQFBgcICQoLDA0ODw==, k2:AAECAAECAAECAAECAAECAA==");
  7. QuickstartUtils.DataGenerator dataGen = new QuickstartUtils.DataGenerator();
  8. List<String> inserts = convertToStringList(dataGen.generateInserts(3));
  9. Dataset<Row> inputDF1 = spark.read().json(jsc.parallelize(inserts, 1));
  10. inputDF1.write().format("org.apache.hudi")
  11. .option("hoodie.table.name", "encryption_table")
  12. .option("hoodie.upsert.shuffle.parallelism","2")
  13. .option("hoodie.insert.shuffle.parallelism","2")
  14. .option("hoodie.delete.shuffle.parallelism","2")
  15. .option("hoodie.bulkinsert.shuffle.parallelism","2")
  16. .mode(SaveMode.Overwrite)
  17. .save("path");
  18. spark.read().format("org.apache.hudi").load("path").select("rider").show();

Reading the table works if configured correctly

  1. +---------+
  2. |rider |
  3. +---------+
  4. |rider-213|
  5. |rider-213|
  6. |rider-213|
  7. +---------+

Read more from Spark docs and Parquet docs.

Note

This feature is currently only available for COW tables due to only Parquet base files present there.