This is an automated email from the ASF dual-hosted git repository.

ulyssesyou pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.4 by this push:
     new faba320702a1 [SPARK-47187][SQL][3.4] Fix hive compress output config 
does not work
faba320702a1 is described below

commit faba320702a185396cfd1def900da5a6cf265108
Author: ulysses-you <[email protected]>
AuthorDate: Wed Feb 28 13:35:38 2024 +0800

    [SPARK-47187][SQL][3.4] Fix hive compress output config does not work
    
    ### What changes were proposed in this pull request?
    
    This pr fixs the issue that `setupHadoopConfForCompression` did not set 
isCompressed as expected due to we implicitly convert ShimFileSinkDesc to 
FileSinkDesc. This issue does not affect master branch since we removed 
ShimFileSinkDesc in https://github.com/apache/spark/pull/40848
    
    ### Why are the changes needed?
    
    To make `hive.exec.compress.output` work as expected.
    
    ### Does this PR introduce _any_ user-facing change?
    
    yes, fix bug
    
    ### How was this patch tested?
    
    manually test
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    no
    
    Closes #45286 from ulysses-you/fix.
    
    Authored-by: ulysses-you <[email protected]>
    Signed-off-by: youxiduo <[email protected]>
---
 .../scala/org/apache/spark/sql/hive/execution/V1WritesHiveUtils.scala  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/V1WritesHiveUtils.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/V1WritesHiveUtils.scala
index 6421dd184ae0..2cfb9d2f15c9 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/V1WritesHiveUtils.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/V1WritesHiveUtils.scala
@@ -21,7 +21,7 @@ import java.util.Locale
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.hive.ql.ErrorMsg
-import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
+import org.apache.hadoop.hive.ql.plan.TableDesc
 
 import org.apache.spark.SparkException
 import org.apache.spark.sql.{AnalysisException, SparkSession}
@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
 import org.apache.spark.sql.execution.datasources.BucketingUtils
+import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc}
 import org.apache.spark.sql.hive.client.HiveClientImpl
 
 trait V1WritesHiveUtils {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to