-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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
Changes from 11 commits
9bbfe6e
48cf108
5dbd3ed
5124f1b
1b087df
05e52b6
0c0f55d
4ab7ecb
3cf0c04
10e5462
2ab2d29
845dda7
e510b48
9229e6f
37fe65e
253b2a2
d60dcd1
b5cd809
26c1c61
9466797
1a8c654
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 |
---|---|---|
|
@@ -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 " + | ||
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. s/use when/used when |
||
"configuration was found through hive or parquet, the precedence would be `compression`, " + | ||
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.
|
||
"`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") | ||
|
@@ -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 " + | ||
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. s/use when/used when 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. 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)) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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 | ||
|
@@ -27,7 +28,7 @@ import org.apache.spark.sql.internal.SQLConf | |
/** | ||
* Options for the Parquet data source. | ||
*/ | ||
private[parquet] class ParquetOptions( | ||
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. Can we revive 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. Yes, It should be revived. Thanks. |
||
class ParquetOptions( | ||
@transient private val parameters: CaseInsensitiveMap[String], | ||
@transient private val sqlConf: SQLConf) | ||
extends Serializable { | ||
|
@@ -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)) | ||
|
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 | ||
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. 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 { | ||
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. This suite does not need |
||
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") | ||
} | ||
} | ||
} |
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.
s/use when/used when