Skip to content
Snippets Groups Projects
Commit d7755cfd authored by hyukjinkwon's avatar hyukjinkwon Committed by Yin Huai
Browse files

[SPARK-14917][SQL] Enable some ORC compressions tests for writing

## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-14917

As it is described in the JIRA, it seems Hive 1.2.1 which Spark uses now supports snappy and none.

So, this PR enables some tests for writing ORC files with compression codes, `SNAPPY` and `NONE`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12699 from HyukjinKwon/SPARK-14917.
parent 09da43d5
No related branches found
No related tags found
No related merge requests found
......@@ -20,8 +20,6 @@ package org.apache.spark.sql.hive.orc
import java.io.File
import java.nio.charset.StandardCharsets
import org.apache.hadoop.hive.conf.HiveConf.ConfVars
import org.apache.hadoop.hive.ql.io.orc.CompressionKind
import org.scalatest.BeforeAndAfterAll
import org.apache.spark.sql._
......@@ -169,39 +167,45 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
}
}
// We only support zlib in Hive 0.12.0 now
test("Default compression options for writing to an ORC file") {
withOrcFile((1 to 100).map(i => (i, s"val_$i"))) { file =>
assertResult(CompressionKind.ZLIB) {
OrcFileOperator.getFileReader(file).get.getCompression
}
// 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 =>
sqlContext.range(0, 10).write
.option("orc.compress", "ZLIB")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("ZLIB" === expectedCompressionKind.name())
}
}
// Following codec is supported in hive-0.13.1, ignore it now
ignore("Other compression options for writing to an ORC file - 0.13.1 and above") {
val data = (1 to 100).map(i => (i, s"val_$i"))
val conf = sqlContext.sessionState.newHadoopConf()
conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "SNAPPY")
withOrcFile(data) { file =>
assertResult(CompressionKind.SNAPPY) {
OrcFileOperator.getFileReader(file).get.getCompression
}
withTempPath { file =>
sqlContext.range(0, 10).write
.option("orc.compress", "SNAPPY")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("SNAPPY" === expectedCompressionKind.name())
}
conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "NONE")
withOrcFile(data) { file =>
assertResult(CompressionKind.NONE) {
OrcFileOperator.getFileReader(file).get.getCompression
}
withTempPath { file =>
sqlContext.range(0, 10).write
.option("orc.compress", "NONE")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("NONE" === expectedCompressionKind.name())
}
}
conf.set(ConfVars.HIVE_ORC_DEFAULT_COMPRESS.varname, "LZO")
withOrcFile(data) { file =>
assertResult(CompressionKind.LZO) {
OrcFileOperator.getFileReader(file).get.getCompression
}
// Following codec is not supported in Hive 1.2.1, ignore it now
ignore("LZO compression options for writing to an ORC file not supported in Hive 1.2.1") {
withTempPath { file =>
sqlContext.range(0, 10).write
.option("orc.compress", "LZO")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert("LZO" === expectedCompressionKind.name())
}
}
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment