You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "Vignesh Nageswaran (Jira)" <ji...@apache.org> on 2022/09/20 13:01:00 UTC

[jira] [Created] (PARQUET-2193) Encrypting only one field in nested field prevents reading of other fields in nested field without keys

Vignesh Nageswaran created PARQUET-2193:
-------------------------------------------

             Summary: Encrypting only one field in nested field prevents reading of other fields in nested field without keys
                 Key: PARQUET-2193
                 URL: https://issues.apache.org/jira/browse/PARQUET-2193
             Project: Parquet
          Issue Type: Bug
          Components: parquet-mr
    Affects Versions: 1.12.0
            Reporter: Vignesh Nageswaran


Hi Team,

While exploring parquet encryption, it is found that, if a field in nested column is encrypted , and If I want to read this parquet directory from other applications which does not have encryption keys to decrypt it, I cannot read the remaining fields of the nested column without keys. 

Example 

`
{code:java}
case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
case class SquareItem(int_column: Int, square_int_column : Double, partitionCol: Int, nestedCol :nestedItem)
`{code}

In the case class `SquareItem` , `nestedCol` field is nested field and I want to encrypt a field `ic` within it. 
 
I also want the footer to be non encrypted , so that I can use the encrypted parquet file by legacy applications. 
 
Encryption is successful, however, when I query the parquet file using spark 3.3.0 without having any configuration for parquet encryption set up , I cannot non encrypted fields of `nestedCol` `sic`. I was expecting that only `nestedCol` `ic` field will not be querable.
 
 
Reproducer. 
Spark 3.3.0 Using Spark-shell 
Downloaded the file [parquet-hadoop-1.12.0-tests.jar|https://repo1.maven.org/maven2/org/apache/parquet/parquet-hadoop/1.12.0/parquet-hadoop-1.12.0-tests.jar] and added it to spark-jars folder
Code to create encrypted data. #  

 
{code:java}
sc.hadoopConfiguration.set("parquet.crypto.factory.class" ,"org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory")

sc.hadoopConfiguration.set("parquet.encryption.kms.client.class" ,"org.apache.parquet.crypto.keytools.mocks.InMemoryKMS")

sc.hadoopConfiguration.set("parquet.encryption.key.list","key1a: BAECAwQFBgcICQoLDA0ODw==, key2a: BAECAAECAAECAAECAAECAA==, keyz: BAECAAECAAECAAECAAECAA==")

sc.hadoopConfiguration.set("parquet.encryption.key.material.store.internally","false")

val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
valpartitionCol = 1
case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
case class SquareItem(int_column: Int, square_int_column : Double, partitionCol: Int, nestedCol :nestedItem)
val dataRange = (1 to 100).toList
val squares = sc.parallelize(dataRange.map(i => new SquareItem(i, scala.math.pow(i,2), partitionCol,nestedItem(i,i))))
squares.toDS().show()
squares.toDS().write.partitionBy("partitionCol").mode("overwrite").option("parquet.encryption.column.keys", "key1a:square_int_column,nestedCol.ic;").option("parquet.encryption.plaintext.footer",true).option("parquet.encryption.footer.key", "keyz").parquet(encryptedParquetPath)
{code}


Code to read the data trying to access non encrypted nested field by opening a new spark-shell
 


{code:java}
val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
spark.sqlContext.read.parquet(encryptedParquetPath).createOrReplaceTempView("test")
spark.sql("select nestedCol.sic from test").show(){code}

As you can see that nestedCol.sic is not encrypted , I was expecting the results, but
I get the below error
 


{code:java}
Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: [square_int_column]. Null File Decryptor
  at org.apache.parquet.hadoop.metadata.EncryptedColumnChunkMetaData.decryptIfNeeded(ColumnChunkMetaData.java:602)
  at org.apache.parquet.hadoop.metadata.ColumnChunkMetaData.getEncodings(ColumnChunkMetaData.java:348)
  at org.apache.parquet.hadoop.ParquetRecordReader.checkDeltaByteArrayProblem(ParquetRecordReader.java:191)
  at org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:177)
  at org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
  at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat.$anonfun$buildReaderWithPartitionValues$1(ParquetFileFormat.scala:375)
  at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.org$apache$spark$sql$execution$datasources$FileScanRDD$$anon$$readCurrentFile(FileScanRDD.scala:209)
  at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:270)
  at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:116)
  at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:460)
  at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(Unknown Source)
  at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
  at org.apache.spark.sql.execution.WholeStageCodegenExec$$anon$1.hasNext(WholeStageCodegenExec.scala:760)
  at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:364)
  at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)
  at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)
  at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
  at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:365)
  at org.apache.spark.rdd.RDD.iterator(RDD.scala:329)
  at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
  at org.apache.spark.scheduler.Task.run(Task.scala:136)
  at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:548)
  at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1504)
  at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:551)
  at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
  at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
  at java.base/java.lang.Thread.run(Thread.java:833){code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)