You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "yaooqinn (via GitHub)" <gi...@apache.org> on 2024/01/18 13:01:04 UTC

[PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

yaooqinn opened a new pull request, #44786:
URL: https://github.com/apache/spark/pull/44786

   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   This PR introduces 2 keys in the form of 'spark.sql.avro.$codecName.level' just like the existing 'spark.sql.avro.deflate.level' for standard and xz codec. W/ this patch, users are able to play the trade-off between the speed and compression ratio when they use AVRO compressed by zstd or xz.
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   Avro supports compression level for deflate, xz and zstd, but we have only supported deflate. 
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   yes, new configurations added
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   new tests
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   no


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1457693863


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -3625,7 +3625,23 @@ object SQLConf {
     .version("2.4.0")
     .intConf
     .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
-    .createWithDefault(Deflater.DEFAULT_COMPRESSION)
+    .createOptional

Review Comment:
   Why do we need to remove the default `Deflater.DEFAULT_COMPRESSION` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun closed pull request #44786: [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files
URL: https://github.com/apache/spark/pull/44786


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1459618369


##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala:
##########
@@ -110,10 +110,12 @@ private[sql] object AvroUtils extends Logging {
           case compressed =>
             job.getConfiguration.setBoolean("mapred.output.compress", true)
             job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
-            if (compressed == DEFLATE) {
-              val deflateLevel = sqlConf.avroDeflateLevel
-              logInfo(s"Compressing Avro output using the $codecName codec at level $deflateLevel")
-              job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel)
+            if (compressed.getSupportCompressionLevel) {
+              val level = sqlConf.getConfString(s"spark.sql.avro.$codecName.level",
+                compressed.getDefaultCompressionLevel.toString)
+              logInfo(s"Compressing Avro output using the $codecName codec at level $level")
+              val s = if (compressed == ZSTANDARD) "zstd" else codecName

Review Comment:
   @beliefer May I ask your reason? For me, it's not required because Avro's real codec name is `zstandard` instead of `zstd` .
   
   **AVRO REPO** https://github.com/apache/avro/blob/8d610fb5c7d3958256801848dbd80d6f9d3c556b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java#L41
   ```
   public static final String ZSTANDARD_CODEC = "zstandard";
   ```
   
   **SPARK REPO**
   https://github.com/apache/spark/blob/39f8e1a5953b5897f893151d24dc585a80c0c8a0/connector/avro/src/main/java/org/apache/spark/sql/avro/AvroCompressionCodec.java#L36



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #44786:
URL: https://github.com/apache/spark/pull/44786#issuecomment-1898811626

   Merged to master for Apache Spark 4.0.0.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1460278740


##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala:
##########
@@ -110,10 +110,12 @@ private[sql] object AvroUtils extends Logging {
           case compressed =>
             job.getConfiguration.setBoolean("mapred.output.compress", true)
             job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
-            if (compressed == DEFLATE) {
-              val deflateLevel = sqlConf.avroDeflateLevel
-              logInfo(s"Compressing Avro output using the $codecName codec at level $deflateLevel")
-              job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel)
+            if (compressed.getSupportCompressionLevel) {
+              val level = sqlConf.getConfString(s"spark.sql.avro.$codecName.level",
+                compressed.getDefaultCompressionLevel.toString)
+              logInfo(s"Compressing Avro output using the $codecName codec at level $level")
+              val s = if (compressed == ZSTANDARD) "zstd" else codecName

Review Comment:
   Got it. `zstd` only used for `avro.mapred.zstd.level`.
   @dongjoon-hyun Thank you for your explanation.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1460278740


##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala:
##########
@@ -110,10 +110,12 @@ private[sql] object AvroUtils extends Logging {
           case compressed =>
             job.getConfiguration.setBoolean("mapred.output.compress", true)
             job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
-            if (compressed == DEFLATE) {
-              val deflateLevel = sqlConf.avroDeflateLevel
-              logInfo(s"Compressing Avro output using the $codecName codec at level $deflateLevel")
-              job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel)
+            if (compressed.getSupportCompressionLevel) {
+              val level = sqlConf.getConfString(s"spark.sql.avro.$codecName.level",
+                compressed.getDefaultCompressionLevel.toString)
+              logInfo(s"Compressing Avro output using the $codecName codec at level $level")
+              val s = if (compressed == ZSTANDARD) "zstd" else codecName

Review Comment:
   Got it. `zstd` only used for `avro.mapred.zstd.level`.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1457695414


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -3625,7 +3625,23 @@ object SQLConf {
     .version("2.4.0")
     .intConf
     .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
-    .createWithDefault(Deflater.DEFAULT_COMPRESSION)
+    .createOptional

Review Comment:
   Oh, nvm. I found it goes to `DEFLATE(DataFileConstants.DEFLATE_CODEC, true, CodecFactory.DEFAULT_DEFLATE_LEVEL),`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "yaooqinn (via GitHub)" <gi...@apache.org>.
yaooqinn commented on PR #44786:
URL: https://github.com/apache/spark/pull/44786#issuecomment-1899516986

   Thank you very much @dongjoon-hyun 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44786:
URL: https://github.com/apache/spark/pull/44786#discussion_r1458650779


##########
connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala:
##########
@@ -110,10 +110,12 @@ private[sql] object AvroUtils extends Logging {
           case compressed =>
             job.getConfiguration.setBoolean("mapred.output.compress", true)
             job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, compressed.getCodecName)
-            if (compressed == DEFLATE) {
-              val deflateLevel = sqlConf.avroDeflateLevel
-              logInfo(s"Compressing Avro output using the $codecName codec at level $deflateLevel")
-              job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel)
+            if (compressed.getSupportCompressionLevel) {
+              val level = sqlConf.getConfString(s"spark.sql.avro.$codecName.level",
+                compressed.getDefaultCompressionLevel.toString)
+              logInfo(s"Compressing Avro output using the $codecName codec at level $level")
+              val s = if (compressed == ZSTANDARD) "zstd" else codecName

Review Comment:
   It seems we should put `logInfo(s"Compressing Avro output using the $codecName codec at level $level")` after val `s = if (compressed == ZSTANDARD) "zstd" else codecName`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org