Skip to content
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

[SPARK-21786][SQL] When acquiring 'compressionCodecClassName' in 'ParquetOptions', parquet.compression needs to be considered. #20076

Closed
wants to merge 21 commits into from
Closed
Show file tree
Hide file tree
Changes from 11 commits
Commits
Show all changes
21 commits
Select commit Hold shift + click to select a range
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 4 additions & 2 deletions docs/sql-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -953,8 +953,10 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession
<td><code>spark.sql.parquet.compression.codec</code></td>
<td>snappy</td>
<td>
Sets the compression codec use when writing Parquet files. Acceptable values include:
uncompressed, snappy, gzip, lzo.
Sets the compression codec use when writing Parquet files. If other compression codec
Copy link
Contributor

Choose a reason for hiding this comment

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

s/use when/used when

configuration was found through hive or parquet, the precedence would be `compression`,
`parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include:
none, uncompressed, snappy, gzip, lzo.
</td>
</tr>
<tr>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -323,11 +323,13 @@ object SQLConf {
.createWithDefault(false)

val PARQUET_COMPRESSION = buildConf("spark.sql.parquet.compression.codec")
.doc("Sets the compression codec use when writing Parquet files. Acceptable values include: " +
"uncompressed, snappy, gzip, lzo.")
.doc("Sets the compression codec use when writing Parquet files. If other compression codec " +
Copy link
Contributor

Choose a reason for hiding this comment

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

s/use when/used when

"configuration was found through hive or parquet, the precedence would be `compression`, " +
Copy link
Member

Choose a reason for hiding this comment

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

Sets the compression codec used when writing Parquet files. If either compression or parquet.compression is specified in the table-specific options/properties, the precedence would be compression, ...

"`parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: " +
"none, uncompressed, snappy, gzip, lzo.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
.checkValues(Set("uncompressed", "snappy", "gzip", "lzo"))
.checkValues(Set("none", "uncompressed", "snappy", "gzip", "lzo"))
.createWithDefault("snappy")

val PARQUET_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.parquet.filterPushdown")
Expand Down Expand Up @@ -364,7 +366,9 @@ object SQLConf {
.createWithDefault(true)

val ORC_COMPRESSION = buildConf("spark.sql.orc.compression.codec")
.doc("Sets the compression codec use when writing ORC files. Acceptable values include: " +
.doc("Sets the compression codec use when writing ORC files. If other compression codec " +
Copy link
Contributor

Choose a reason for hiding this comment

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

s/use when/used when

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thank you. I had fixed them.

"configuration was found through hive or ORC, the precedence would be `compression`, " +
"`orc.compress`, `spark.sql.orc.compression.codec`. Acceptable values include: " +
"none, uncompressed, snappy, zlib, lzo.")
.stringConf
.transform(_.toLowerCase(Locale.ROOT))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ class OrcFileFormat

conf.set(MAPRED_OUTPUT_SCHEMA.getAttribute, dataSchema.catalogString)

conf.set(COMPRESS.getAttribute, orcOptions.compressionCodec)
conf.set(COMPRESS.getAttribute, orcOptions.compressionCodecName)

conf.asInstanceOf[JobConf]
.setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ class OrcOptions(
* Compression codec to use.
* Acceptable values are defined in [[shortOrcCompressionCodecNames]].
*/
val compressionCodec: String = {
val compressionCodecName: String = {
// `compression`, `orc.compress`(i.e., OrcConf.COMPRESS), and `spark.sql.orc.compression.codec`
// are in order of precedence from highest to lowest.
val orcCompressionConf = parameters.get(COMPRESS.getAttribute)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ class ParquetFileFormat
sparkSession.sessionState.conf.parquetOutputTimestampType.toString)

// Sets compression scheme
conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName)
conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecName)

// SPARK-15719: Disables writing Parquet summary files by default.
if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.parquet

import java.util.Locale

import org.apache.parquet.hadoop.ParquetOutputFormat
import org.apache.parquet.hadoop.metadata.CompressionCodecName

import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
Expand All @@ -27,7 +28,7 @@ import org.apache.spark.sql.internal.SQLConf
/**
* Options for the Parquet data source.
*/
private[parquet] class ParquetOptions(
Copy link
Member

Choose a reason for hiding this comment

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

Can we revive private[parquet]?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, It should be revived. Thanks.

class ParquetOptions(
@transient private val parameters: CaseInsensitiveMap[String],
@transient private val sqlConf: SQLConf)
extends Serializable {
Expand All @@ -41,9 +42,16 @@ private[parquet] class ParquetOptions(
* Compression codec to use. By default use the value specified in SQLConf.
* Acceptable values are defined in [[shortParquetCompressionCodecNames]].
*/
val compressionCodecClassName: String = {
val codecName = parameters.getOrElse("compression",
sqlConf.parquetCompressionCodec).toLowerCase(Locale.ROOT)
val compressionCodecName: String = {
// `compression`, `parquet.compression`(i.e., ParquetOutputFormat.COMPRESSION), and
// `spark.sql.parquet.compression.codec`
// are in order of precedence from highest to lowest.
val parquetCompressionConf = parameters.get(ParquetOutputFormat.COMPRESSION)
val codecName = parameters
.get("compression")
.orElse(parquetCompressionConf)
.getOrElse(sqlConf.parquetCompressionCodec)
.toLowerCase(Locale.ROOT)
if (!shortParquetCompressionCodecNames.contains(codecName)) {
val availableCodecs =
shortParquetCompressionCodecNames.keys.map(_.toLowerCase(Locale.ROOT))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -134,29 +134,29 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll {
test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
val conf = spark.sessionState.conf
val option = new OrcOptions(Map(COMPRESS.getAttribute.toUpperCase(Locale.ROOT) -> "NONE"), conf)
assert(option.compressionCodec == "NONE")
assert(option.compressionCodecName == "NONE")
}

test("SPARK-21839: Add SQL config for ORC compression") {
val conf = spark.sessionState.conf
// Test if the default of spark.sql.orc.compression.codec is snappy
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "SNAPPY")
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodecName == "SNAPPY")

// OrcOptions's parameters have a higher priority than SQL configuration.
// `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec`
withSQLConf(SQLConf.ORC_COMPRESSION.key -> "uncompressed") {
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "NONE")
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodecName == "NONE")
val map1 = Map(COMPRESS.getAttribute -> "zlib")
val map2 = Map(COMPRESS.getAttribute -> "zlib", "compression" -> "lzo")
assert(new OrcOptions(map1, conf).compressionCodec == "ZLIB")
assert(new OrcOptions(map2, conf).compressionCodec == "LZO")
assert(new OrcOptions(map1, conf).compressionCodecName == "ZLIB")
assert(new OrcOptions(map2, conf).compressionCodecName == "LZO")
}

// Test all the valid options of spark.sql.orc.compression.codec
Seq("NONE", "UNCOMPRESSED", "SNAPPY", "ZLIB", "LZO").foreach { c =>
withSQLConf(SQLConf.ORC_COMPRESSION.key -> c) {
val expected = if (c == "UNCOMPRESSED") "NONE" else c
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == expected)
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodecName == expected)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -768,7 +768,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") {
val option = new ParquetOptions(Map("Compression" -> "uncompressed"), spark.sessionState.conf)
assert(option.compressionCodecClassName == "UNCOMPRESSED")
assert(option.compressionCodecName == "UNCOMPRESSED")
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable

val configuration = job.getConfiguration

configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec)
configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodecName)
configuration match {
case conf: JobConf =>
conf.setOutputFormat(classOf[OrcOutputFormat])
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.hive
Copy link
Member

Choose a reason for hiding this comment

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

Move it to sql/core.


import org.apache.parquet.hadoop.ParquetOutputFormat

import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions
import org.apache.spark.sql.hive.test.TestHiveSingleton
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SQLTestUtils

class CompressionCodecSuite extends TestHiveSingleton with SQLTestUtils {
Copy link
Member

Choose a reason for hiding this comment

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

This suite does not need TestHiveSingleton .

test("Test `spark.sql.parquet.compression.codec` config") {
Seq("NONE", "UNCOMPRESSED", "SNAPPY", "GZIP", "LZO").foreach { c =>
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> c) {
val expected = if (c == "NONE") "UNCOMPRESSED" else c
val option = new ParquetOptions(Map.empty[String, String], spark.sessionState.conf)
assert(option.compressionCodecName == expected)
}
}
}

test("[SPARK-21786] Test Acquiring 'compressionCodecClassName' for parquet in right order.") {
// When "compression" is configured, it should be the first choice.
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") {
val props = Map("compression" -> "uncompressed", ParquetOutputFormat.COMPRESSION -> "gzip")
val option = new ParquetOptions(props, spark.sessionState.conf)
assert(option.compressionCodecName == "UNCOMPRESSED")
}

// When "compression" is not configured, "parquet.compression" should be the preferred choice.
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") {
val props = Map(ParquetOutputFormat.COMPRESSION -> "gzip")
val option = new ParquetOptions(props, spark.sessionState.conf)
assert(option.compressionCodecName == "GZIP")
}

// When both "compression" and "parquet.compression" are not configured,
// spark.sql.parquet.compression.codec should be the right choice.
withSQLConf(SQLConf.PARQUET_COMPRESSION.key -> "snappy") {
val props = Map.empty[String, String]
val option = new ParquetOptions(props, spark.sessionState.conf)
assert(option.compressionCodecName == "SNAPPY")
}
}
}