[
https://issues.apache.org/jira/browse/SPARK-56154?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Anitesh Minj updated SPARK-56154:
---------------------------------
Attachment: target_file.parquet
> [PARQUET] BINARY(ENUM) logical annotation lost during reading and writing
> back Parquet files - ENUM silently converted to STRING causing schema and
> data corruption
> ---------------------------------------------------------------------------------------------------------------------------------------------------------------------
>
> Key: SPARK-56154
> URL: https://issues.apache.org/jira/browse/SPARK-56154
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 2.4.0, 3.3.0, 3.3.2
> Reporter: Anitesh Minj
> Priority: Major
> Attachments: enum_source.parquet, target_file.parquet
>
>
> When Spark reads a Parquet file containing a column with *BINARY physical
> type* and *ENUM logical annotation*, and writes it back to a new Parquet
> file, the ENUM logical annotation is silently lost and replaced with STRING
> annotation.
>
> The ENUM field stores valid UTF8/ASCII bytes representing a fixed set of
> string values (e.g. "CLUBS"). Parquet viewers and downstream consumers read
> these as raw byte maps:
> {"0":67,"1":76,"2":85,"3":66,"4":83}
> Since Spark does not have a native Enum type, it treats the ENUM-annotated
> BINARY field as a String, so the same data is represented as:-
> "CLUBS"
>
> Steps to reproduce:-
> 1. Check the Parquet schema after reading it using ParquetFileReader:
> {code:java}
> import org.apache.parquet.hadoop.ParquetFileReader
> import org.apache.parquet.hadoop.util.HadoopInputFile
> import org.apache.hadoop.fs.Path
> import org.apache.hadoop.conf.Configuration
> val conf = spark.sparkContext.hadoopConfiguration
> val path = new Path("/tmp/enum-source.parquet")
> val inputFile = HadoopInputFile.fromPath(path, conf)
> val sourceReader = ParquetFileReader.open(inputFile)
> println(sourceReader.getFileMetaData.getSchema)
> message com.test.avro.MyClass {
> required binary suit (ENUM);
> }{code}
>
>
> 2. Read the attached enum_source.parquet file using Spark
> {code:java}
> import org.apache.spark.sql.functions._
> import org.apache.spark.sql.types._
> val sqlContext = spark.sqlContext
> val sourceDF = sqlContext.read.parquet("/tmp/enum-source.parquet") {code}
>
> 3. Write the dataframe sourceDF back to a parquet file
> {code:java}
> sourceDF.write.option("compression",
> "none").mode("overwrite").parquet("/tmp/enum-target"){code}
>
> 4. Read the written Parquet file schema using ParquetFileReader
> {code:java}
> import org.apache.parquet.hadoop.ParquetFileReader
> import org.apache.parquet.hadoop.util.HadoopInputFile
> import org.apache.hadoop.fs.Path
> import org.apache.hadoop.conf.Configuration
> val conf = spark.sparkContext.hadoopConfiguration
> val targetPath = new Path("/tmp/enum-target/${target_file_name}.parquet")
> val targetInputFile = HadoopInputFile.fromPath(targetPath, conf)
> val targetReader = ParquetFileReader.open(targetInputFile)
> println(targetReader.getFileMetaData.getSchema)
> message spark_schema {
> optional binary suit (STRING);
> }{code}
>
>
> As we can clearly see, the schema has changed and the logical annotation has
> been changed from ENUM to STRING.
> This leads to data corruption as well.
> Source data (when read using a Parquet viewer):
> {"suit":{color:#FF0000}{"0":83,"1":80,"2":65,"3":68,"4":69,"5":83}{color}}
> Target data (after writing the DataFrame sourceDF):-
> {"suit":"{color:#FF0000}SPADES"{color}}
> *Expected Behavior:-*
> Source Schema:
> {code:java}
> required binary suit (ENUM) {code}
> Target Schema:
> {code:java}
> required binary suit (ENUM) {code}
> Source Data:
> {code:java}
> {"suit":{"0":83,"1":80,"2":65,"3":68,"4":69,"5":83}} {code}
> Target Data:
> {code:java}
> {"suit":{"0":83,"1":80,"2":65,"3":68,"4":69,"5":83}} {code}
>
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]