-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-24881][SQL] New Avro option - compression #21837
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
Conversation
|
Test build #93404 has finished for PR 21837 at commit
|
|
jenkins, retest this, please |
|
Test build #93405 has finished for PR 21837 at commit
|
| .avro(snappyDir) | ||
|
|
||
| val uncompressSize = FileUtils.sizeOfDirectory(new File(uncompressDir)) | ||
| val deflateSize = FileUtils.sizeOfDirectory(new File(deflateDir)) |
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.
is there any or easy way to check the metadata for compression level?
|
Test build #93408 has finished for PR 21837 at commit
|
|
|
||
| case "deflate" => | ||
| val deflateLevel = spark.conf.get( | ||
| AVRO_DEFLATE_LEVEL, Deflater.DEFAULT_COMPRESSION.toString).toInt |
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.
Deflater.DEFAULT_COMPRESSION is -1 here. Why change the default value to -6 in SQLConf?
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.
I changed it because I didn't have any ideas what -1 means. Is it closer to best compression 9 or fast compression 1? Probably the level is eventually set in zlib in which -1 means 6. @gengliangwang From your point of view, -1 means better or faster compression?
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.
If the compression algorithm changes in the feature, the default value may not be 6. But if we use DEFAULT_COMPRESSION, it will still be equivalent to the new default value. So I suggest we still use Deflater.DEFAULT_COMPRESSION instead of any specific number.
| val deflateDir = s"$dir/deflate" | ||
| val snappyDir = s"$dir/snappy" | ||
|
|
||
| val df = spark.read.avro(testAvro) |
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.
I am removing all the .avro method: #21841
Could you change it to ".format("avro").load" or ".format("avro").save"? Thanks!
# Conflicts: # external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
@HyukjinKwon I am not sure the level exists in the metadata. At least |
|
Test build #93461 has finished for PR 21837 at commit
|
| .save(snappyDir) | ||
|
|
||
| val uncompressSize = FileUtils.sizeOfDirectory(new File(uncompressDir)) | ||
| val deflateSize = FileUtils.sizeOfDirectory(new File(deflateDir)) |
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.
Thank you, @MaxGekk. Can we then check the type of compression at least avro.codec deflate?
| * config `spark.sql.avro.deflate.level` is used by default. For other compressions, the default | ||
| * value is `6`. | ||
| */ | ||
| val compressionLevel: Int = { |
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.
Can we don't expose this as an option for now? IIUC, this compression level only applies to deflate, right? Also, this option looks not for keeping the same options from the thrid party as well.
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.
I added the option keeping in mind other compression codecs can be added in the future, for example zstandard. For those codecs, the level could be useful too. Another point is specifying compression level together with compression codec in Avro options looks more natural comparing to SQL global settings:
df.write
.options(Map("compression" -> "deflate", "compressionLevel" -> "9"))
.format("avro")
.save(deflateDir)
vs
spark.conf.set("spark.sql.avro.deflate.level", "9")
df.write
.option("compression", "deflate"))
.format("avro")
.save(deflateDir)
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.
Yea, I know that could be useful in some ways but I was thinking we should better not add this just for now. Thing is, it sounds currently too specific to one compression option in Avro for now .. There are many options to expose in, for example in CSV datasource too in this way ..
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.
Also, to be honest, I wonder users would want to change compression level often ..
| val AVRO_COMPRESSION_CODEC = buildConf("spark.sql.avro.compression.codec") | ||
| .doc("Compression codec used in writing of AVRO files.") | ||
| .stringConf | ||
| .createWithDefault("snappy") |
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.
can we .checkValues(Set( too?
| val AVRO_DEFLATE_LEVEL = buildConf("spark.sql.avro.deflate.level") | ||
| .doc("Compression level for the deflate codec used in writing of AVRO files. " + | ||
| "Valid value must be in the range of from 1 to 9 inclusive. " + | ||
| "The default value is -1 which corresponds to 6 level in the current implementation.") |
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.
can we do the check like checkValue(_ => -1, ...) here
|
|
||
| val AVRO_DEFLATE_LEVEL = buildConf("spark.sql.avro.deflate.level") | ||
| .doc("Compression level for the deflate codec used in writing of AVRO files. " + | ||
| "Valid value must be in the range of from 1 to 9 inclusive. " + |
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.
This can be -1 right (https://www.zlib.net/manual.html)?
# Conflicts: # external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
| .createWithDefault(20) | ||
|
|
||
| val AVRO_COMPRESSION_CODEC = buildConf("spark.sql.avro.compression.codec") | ||
| .doc("Compression codec used in writing of AVRO files.") |
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.
Document the default value?
|
Test build #93636 has finished for PR 21837 at commit
|
HyukjinKwon
left a comment
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.
LGTM
|
Test build #93632 has finished for PR 21837 at commit
|
| @transient val parameters: CaseInsensitiveMap[String], | ||
| @transient val conf: Configuration) extends Logging with Serializable { | ||
| @transient val conf: Configuration, | ||
| @transient val sqlConf: SQLConf) extends Logging with Serializable { |
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.
We may just get SQLConf by calling SQLConf.get without passing it in.
| test("write with compression") { | ||
| test("write with compression - sql configs") { | ||
| withTempPath { dir => | ||
| val AVRO_COMPRESSION_CODEC = "spark.sql.avro.compression.codec" |
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.
This can use SQLConf.AVRO_COMPRESSION_CODEC.key now.
| test("write with compression - sql configs") { | ||
| withTempPath { dir => | ||
| val AVRO_COMPRESSION_CODEC = "spark.sql.avro.compression.codec" | ||
| val AVRO_DEFLATE_LEVEL = "spark.sql.avro.deflate.level" |
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.
ditto.
|
Test build #93634 has finished for PR 21837 at commit
|
|
Test build #93637 has finished for PR 21837 at commit
|
|
Test build #93658 has finished for PR 21837 at commit
|
|
jenkins, retest this, please |
|
Test build #93659 has finished for PR 21837 at commit
|
|
Merged to master. |
In the PR, I added new option for Avro datasource - `compression`. The option allows to specify compression codec for saved Avro files. This option is similar to `compression` option in another datasources like `JSON` and `CSV`. Also I added the SQL configs `spark.sql.avro.compression.codec` and `spark.sql.avro.deflate.level`. I put the configs into `SQLConf`. If the `compression` option is not specified by an user, the first SQL config is taken into account. I added new test which read meta info from written avro files and checks `avro.codec` property. Author: Maxim Gekk <[email protected]> Closes apache#21837 from MaxGekk/avro-compression. (cherry picked from commit 0a0f68b)
What changes were proposed in this pull request?
In the PR, I added new option for Avro datasource -
compression. The option allows to specify compression codec for saved Avro files. This option is similar tocompressionoption in another datasources likeJSONandCSV.Also I added the SQL configs
spark.sql.avro.compression.codecandspark.sql.avro.deflate.level. I put the configs intoSQLConf. If thecompressionoption is not specified by an user, the first SQL config is taken into account.How was this patch tested?
I added new test which read meta info from written avro files and checks
avro.codecproperty.