Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -20,8 +20,7 @@ package org.apache.spark.sql.hive.orc
/**
* Options for the ORC data source.
*/
private[orc] class OrcOptions(
@transient private val parameters: Map[String, String])
private[orc] class OrcOptions(@transient private val parameters: Map[String, String])
extends Serializable {

import OrcOptions._
Expand All @@ -31,7 +30,8 @@ private[orc] class OrcOptions(
* Acceptable values are defined in [[shortOrcCompressionCodecNames]].
*/
val compressionCodec: String = {
val codecName = parameters.getOrElse("compression", "snappy").toLowerCase
val codecName = parameters.getOrElse(
"compression", parameters.getOrElse("orc.compress", "snappy")).toLowerCase
Copy link
Contributor

Choose a reason for hiding this comment

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

use OrcRelation.ORC_COMPRESSION (since we have a val defined)? Let's add comments to explain what we are doing (we should mention that orc.compress is a ORC conf and which conf will take precedence). Also, will the following lines look better?

    val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION)
    val codecName = parameters
      .get("compression")
      .orElse(orcCompressionConf)
      .getOrElse("snappy")

Copy link
Member Author

Choose a reason for hiding this comment

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

Sure, Thanks for the cleaner snippet!

if (!shortOrcCompressionCodecNames.contains(codecName)) {
val availableCodecs = shortOrcCompressionCodecNames.keys.map(_.toLowerCase)
throw new IllegalArgumentException(s"Codec [$codecName] " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,29 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
}
}

test("SPARK-16610: Respect orc.compress option when compression is unset") {
// Respect `orc.compress`.
withTempPath { file =>
spark.range(0, 10).write
.option("orc.compress", "ZLIB")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("ZLIB" === expectedCompressionKind.name())
}

// `compression` overrides `orc.compress`.
withTempPath { file =>
spark.range(0, 10).write
.option("compression", "ZLIB")
.option("orc.compress", "SNAPPY")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("ZLIB" === expectedCompressionKind.name())
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

nice. Thank you for adding this test.


// Hive supports zlib, snappy and none for Hive 1.2.1.
test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") {
withTempPath { file =>
Expand Down