Skip to content

Commit

Permalink
[GLUTEN-8455][VL] Support encrypted parquet fallback for 3.5 (#8560)
Browse files Browse the repository at this point in the history
  • Loading branch information
ArnavBalyan authored Jan 22, 2025
1 parent e8c9bb5 commit 0f4489a
Show file tree
Hide file tree
Showing 2 changed files with 30 additions and 11 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -110,9 +110,7 @@ class ParquetEncryptionDetectionSuite extends GlutenQueryTest {
fs.listFiles(new Path(path), false).next()
}

testWithSpecifiedSparkVersion(
"Detect encrypted Parquet with encrypted footer",
Array("3.2", "3.3", "3.4")) {
test("Detect encrypted Parquet with encrypted footer") {
withTempDir {
tempDir =>
val filePath = s"${tempDir.getAbsolutePath}/encrypted_footer.parquet"
Expand All @@ -134,9 +132,7 @@ class ParquetEncryptionDetectionSuite extends GlutenQueryTest {
}
}

testWithSpecifiedSparkVersion(
"Detect encrypted Parquet without encrypted footer (plaintext footer)",
Array("3.2", "3.3", "3.4")) {
test("Detect encrypted Parquet without encrypted footer (plaintext footer)") {
withTempDir {
tempDir =>
val filePath = s"${tempDir.getAbsolutePath}/plaintext_footer.parquet"
Expand All @@ -158,9 +154,7 @@ class ParquetEncryptionDetectionSuite extends GlutenQueryTest {
}
}

testWithSpecifiedSparkVersion(
"Detect plain (unencrypted) Parquet file",
Array("3.2", "3.3", "3.4")) {
test("Detect plain (unencrypted) Parquet file") {
withTempDir {
tempDir =>
val filePath = s"${tempDir.getAbsolutePath}/plain.parquet"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ package org.apache.gluten.sql.shims.spark35

import org.apache.gluten.expression.{ExpressionNames, Sig}
import org.apache.gluten.sql.shims.{ShimDescriptor, SparkShims}
import org.apache.gluten.utils.ExceptionUtils

import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
Expand Down Expand Up @@ -57,6 +58,10 @@ import org.apache.spark.storage.{BlockId, BlockManagerId}

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, LocatedFileStatus, Path}
import org.apache.parquet.crypto.ParquetCryptoRuntimeException
import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.metadata.FileMetaData.EncryptionType
import org.apache.parquet.schema.MessageType

import java.time.ZoneOffset
Expand Down Expand Up @@ -554,7 +559,27 @@ class Spark35Shims extends SparkShims {
override def isParquetFileEncrypted(
fileStatus: LocatedFileStatus,
conf: Configuration): Boolean = {
// TODO: Support will be added (https://github.com/apache/incubator-gluten/pull/8501)
return false
try {
val footer =
ParquetFileReader.readFooter(conf, fileStatus.getPath, ParquetMetadataConverter.NO_FILTER)
val fileMetaData = footer.getFileMetaData
fileMetaData.getEncryptionType match {
// UNENCRYPTED file has a plaintext footer and no file encryption,
// We can leverage file metadata for this check and return unencrypted.
case EncryptionType.UNENCRYPTED =>
false
// PLAINTEXT_FOOTER has a plaintext footer however the file is encrypted.
// In such cases, read the footer and use the metadata for encryption check.
case EncryptionType.PLAINTEXT_FOOTER =>
true
case _ =>
false
}
} catch {
// Both footer and file are encrypted, return false.
case e: Exception if ExceptionUtils.hasCause(e, classOf[ParquetCryptoRuntimeException]) =>
true
case e: Exception => false
}
}
}

0 comments on commit 0f4489a

Please sign in to comment.