-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-46759][SQL][AVRO] Codec xz and zstandard support compression level for avro files #44786
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -3625,7 +3625,23 @@ object SQLConf { | |
| .version("2.4.0") | ||
| .intConf | ||
| .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION) | ||
| .createWithDefault(Deflater.DEFAULT_COMPRESSION) | ||
| .createOptional | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we need to remove the default
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, nvm. I found it goes to |
||
|
|
||
| val AVRO_XZ_LEVEL = buildConf("spark.sql.avro.zx.level") | ||
| .doc("Compression level for the xz codec used in writing of AVRO files. " + | ||
| "Valid value must be in the range of from 1 to 9 inclusive " + | ||
| "The default value is 6.") | ||
| .version("4.0.0") | ||
| .intConf | ||
| .checkValue(v => v > 0 && v <= 9, "The value must be in the range of from 1 to 9 inclusive.") | ||
| .createOptional | ||
|
|
||
| val AVRO_ZSTANDARD_LEVEL = buildConf("spark.sql.avro.zstandard.level") | ||
| .doc("Compression level for the zstandard codec used in writing of AVRO files. " + | ||
| "The default value is 3.") | ||
| .version("4.0.0") | ||
| .intConf | ||
| .createOptional | ||
|
|
||
| val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") | ||
| .internal() | ||
|
|
@@ -5421,8 +5437,6 @@ class SQLConf extends Serializable with Logging with SqlApiConf { | |
|
|
||
| def avroCompressionCodec: String = getConf(SQLConf.AVRO_COMPRESSION_CODEC) | ||
|
|
||
| def avroDeflateLevel: Int = getConf(SQLConf.AVRO_DEFLATE_LEVEL) | ||
|
|
||
| def replaceDatabricksSparkAvroEnabled: Boolean = | ||
| getConf(SQLConf.LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED) | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems we should put
logInfo(s"Compressing Avro output using the $codecName codec at level $level")after vals = if (compressed == ZSTANDARD) "zstd" else codecName.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@beliefer May I ask your reason? For me, it's not required because Avro's real codec name is
zstandardinstead ofzstd.AVRO REPO https://github.com/apache/avro/blob/8d610fb5c7d3958256801848dbd80d6f9d3c556b/lang/java/avro/src/main/java/org/apache/avro/file/DataFileConstants.java#L41
SPARK REPO
spark/connector/avro/src/main/java/org/apache/spark/sql/avro/AvroCompressionCodec.java
Line 36 in 39f8e1a
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Got it.
zstdonly used foravro.mapred.zstd.level.@dongjoon-hyun Thank you for your explanation.