[ https://issues.apache.org/jira/browse/SPARK-49394?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Wei Zheng resolved SPARK-49394. ------------------------------- Resolution: Fixed > hive.exec.compress.output not working as expected > ------------------------------------------------- > > Key: SPARK-49394 > URL: https://issues.apache.org/jira/browse/SPARK-49394 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 3.4.0 > Reporter: Eugen Stoianovici > Priority: Major > Fix For: 3.4.3 > > > reproduction: > {code:java} > import org.apache.spark.sql.SaveMode > spark.conf.set("hive.exec.compress.output", "true") > spark.conf.set("mapred.output.compression.codec", > "org.apache.hadoop.io.compress.BZip2Codec") > spark.conf.set("mapred.output.compression.type", "BLOCK") > val data = Seq( > (1, "Test1", 100), > (2, "Test2", 200), > (3, "Test3", 300), > (4, "Test4", 400), > (5, "Test5", 500) > ) > val data = spark.createDataFrame(data).toDF("a", "b", "c") > val databaseName = "test_database" > val tableName = "test_table" > spark.sql(s"DROP DATABASE IF EXISTS $databaseName CASCADE") > spark.sql(s"CREATE DATABASE $databaseName") > data.write.format("hive") > .mode(SaveMode.Overwrite) > .saveAsTable(s"$databaseName.$tableName") > {code} > Expectation/Behaviour in Spark != 3.4: > {noformat} > $ hadoop fs -ls /user/spark/warehouse/test_database.db/test_table > Found 5 items > -rwxr-xr-x 1 hadoop spark 53 2024-08-23 15:39 > /user/spark/warehouse/test_database.db/test_table/part-00000-c2f5bb2d-afbe-4f28-bc5b-b076211c3c15-c000.bz2 > -rwxr-xr-x 1 hadoop spark 53 2024-08-23 15:39 > /user/spark/warehouse/test_database.db/test_table/part-00001-c2f5bb2d-afbe-4f28-bc5b-b076211c3c15-c000.bz2 > ... > {noformat} > In Spark 3.4 the {{.bz2}} extension is missing and the files aren't > compressed. > > This bug was introduced in 3.4.0 by the refactoring in SPARK-41708 - pull > request #39277 and "accidentally" fixed in 3.5 by SPARK-43186 > The patch which breaks the functionality relies on the "pass by pointer" > nature of Scala/Java to populate the compression parameters in an object > {code} > val fileSinkConf = new FileSinkDesc(hiveTempPath.externalTempPath.toString, > tableDesc, false) > setupHadoopConfForCompression(fileSinkConf, hadoopConf, sparkSession) > {code} > However, in Spark 3.4, the {{fileSinkConf}} is wrapped in a shim which makes > a copy when translating a {{ShimFileSinkDesc}} to a {{FileSinkDesc}}. This > means that the {{setupHadoopConfForCompression}} operates on the copy rather > than the object being used afterwards. > This is fixed in 3.5 because the shim (and thus the copy) was removed. -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org