Skip to content
Closed
Show file tree
Hide file tree
Changes from all 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 @@ -508,7 +508,7 @@ abstract class OrcQueryTest extends OrcTest {
conf.setBoolean("hive.io.file.read.all.columns", false)

val orcRecordReader = {
val file = new File(path).listFiles().find(_.getName.endsWith(".snappy.orc")).head
Copy link
Member Author

Choose a reason for hiding this comment

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

This test case should be independent from the default codec, snappy.

val file = new File(path).listFiles().find(_.getName.endsWith(".orc")).head
val split = new FileSplit(new Path(file.toURI), 0, file.length, Array.empty[String])
val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0)
val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -332,8 +332,9 @@ abstract class OrcSuite

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.name())
// Test if the default of spark.sql.orc.compression.codec is used.
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec ==
SQLConf.ORC_COMPRESSION.defaultValueString.toUpperCase(Locale.ROOT))

// OrcOptions's parameters have a higher priority than SQL configuration.
// `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec`
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -107,16 +107,6 @@ class OrcHadoopFsRelationSuite extends HadoopFsRelationTest {
checkAnswer(df, copyDf)
}
}

test("Default compression codec is snappy for ORC compression") {
withTempPath { file =>
spark.range(0, 10).write
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
assert(OrcCompressionCodec.SNAPPY.name() === expectedCompressionKind.name())
}
}
}

class HiveOrcHadoopFsRelationSuite extends OrcHadoopFsRelationSuite {
Expand Down