Skip to content

Conversation

@yaooqinn
Copy link
Member

What changes were proposed in this pull request?

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?

Avro supports compression level for deflate, xz and zstd, but we have only supported deflate.

Does this PR introduce any user-facing change?

yes, new configurations added

How was this patch tested?

new tests

Was this patch authored or co-authored using generative AI tooling?

no

.intConf
.checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
.createWithDefault(Deflater.DEFAULT_COMPRESSION)
.createOptional
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need to remove the default Deflater.DEFAULT_COMPRESSION here?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is really a nice improvement. Thank you, @yaooqinn .

@dongjoon-hyun
Copy link
Member

Merged to master for Apache Spark 4.0.0.

@yaooqinn
Copy link
Member Author

Thank you very much @dongjoon-hyun

@yaooqinn yaooqinn deleted the SPARK-46759 branch January 19, 2024 01:58
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
Copy link
Contributor

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 val s = if (compressed == ZSTANDARD) "zstd" else codecName.

Copy link
Member

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 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

ZSTANDARD(DataFileConstants.ZSTANDARD_CODEC, true, CodecFactory.DEFAULT_ZSTANDARD_LEVEL);

Copy link
Contributor

@beliefer beliefer Jan 20, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants