Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -32,6 +32,7 @@ import org.apache.hadoop.io.{NullWritable, Writable}
import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter}
import org.apache.hadoop.mapreduce._
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
import org.apache.orc.OrcConf.COMPRESS

import org.apache.spark.TaskContext
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -72,7 +73,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable

val configuration = job.getConfiguration

configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec)
configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec)
configuration match {
case conf: JobConf =>
conf.setOutputFormat(classOf[OrcOutputFormat])
Expand All @@ -93,7 +94,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable

override def getFileExtension(context: TaskAttemptContext): String = {
val compressionExtension: String = {
val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION)
val name = context.getConfiguration.get(COMPRESS.getAttribute)
OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "")
}

Expand Down Expand Up @@ -256,9 +257,6 @@ private[orc] class OrcOutputWriter(
}

private[orc] object OrcRelation extends HiveInspectors {
Copy link
Member

Choose a reason for hiding this comment

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

@dongjoon-hyun, mind changing this to OrcFileFormat while we are here (see #14529)?

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. No problem. Thank you for review, @HyukjinKwon !

// The references of Hive's classes will be minimized.
val ORC_COMPRESSION = "orc.compress"
Copy link
Member

@viirya viirya Oct 16, 2017

Choose a reason for hiding this comment

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

We have documented orc.compress as option name explicitly in

* `orc.compress` and `spark.sql.parquet.compression.codec`. If `orc.compress` is given,

Now we depends on the configuration name from an external library. But I think the configuration name should not be changed at all. So looks should be fine.

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Oct 16, 2017

Choose a reason for hiding this comment

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

Yep. I agree. I don't think Apache ORC changes this in the future since this is a primitive configuration.
BTW, Thank you for pointing this doc. I'll fix some typos here.

Copy link
Member

Choose a reason for hiding this comment

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

Oh, right, the parquet.


// This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public.
private[orc] val SARG_PUSHDOWN = "sarg.pushdown"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.orc

import java.util.Locale

import org.apache.orc.OrcConf.COMPRESS

import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
import org.apache.spark.sql.internal.SQLConf

Expand All @@ -42,7 +44,7 @@ private[orc] class OrcOptions(
val compressionCodec: String = {
// `compression`, `orc.compress`, and `spark.sql.orc.compression.codec` are
// in order of precedence from highest to lowest.
Copy link
Member

Choose a reason for hiding this comment

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

Please update the comment.

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Oct 16, 2017

Choose a reason for hiding this comment

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

Thank you for review, @gatorsmile .
The name orc.compress and precedence order is not changed. Which part do you want to change?

Copy link
Member

Choose a reason for hiding this comment

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

COMPRESS.getAttribute is not in our code base. Please change it to

// compression, orc.compress(i.e., OrcConf.COMPRESS), and spark.sql.orc.compression.codec are

Copy link
Member Author

Choose a reason for hiding this comment

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

I see.

val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION)
val orcCompressionConf = parameters.get(COMPRESS.getAttribute)
val codecName = parameters
.get("compression")
.orElse(orcCompressionConf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.sql.Timestamp

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.hive.ql.io.orc.{OrcStruct, SparkOrcNewRecordReader}
import org.apache.orc.OrcConf.COMPRESS
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.sql._
Expand Down Expand Up @@ -180,7 +181,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
// Respect `orc.compress`.
Copy link
Member

Choose a reason for hiding this comment

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

Shall we change this comment too?

Copy link
Member

Choose a reason for hiding this comment

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

Also the test name Respect orc.compress option when compression is unset.

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. Thank you for review, @viirya .

withTempPath { file =>
spark.range(0, 10).write
.option("orc.compress", "ZLIB")
.option(COMPRESS.getAttribute, "ZLIB")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
Expand All @@ -191,7 +192,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
withTempPath { file =>
spark.range(0, 10).write
.option("compression", "ZLIB")
.option("orc.compress", "SNAPPY")
.option(COMPRESS.getAttribute, "SNAPPY")
.orc(file.getCanonicalPath)
val expectedCompressionKind =
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.orc

import java.io.File

import org.apache.orc.OrcConf.COMPRESS
import org.scalatest.BeforeAndAfterAll

Copy link
Member

Choose a reason for hiding this comment

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

Another test in OrcSourceSuite:

assert(new OrcOptions(Map("Orc.Compress" -> "NONE"), conf).compressionCodec == "NONE")

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Oct 16, 2017

Choose a reason for hiding this comment

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

This is case sensitivity test case. We should not change this.

Copy link
Member

Choose a reason for hiding this comment

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

Oh, I was thinking something like COMPRESS.getAttribute.toUpperCase.

Copy link
Member Author

Choose a reason for hiding this comment

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

If you want, no problem at all. :)

Copy link
Member

Choose a reason for hiding this comment

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

Just like to be consistent. Thanks. :)

Copy link
Member Author

Choose a reason for hiding this comment

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

Yep!

import org.apache.spark.sql.{QueryTest, Row}
Expand Down Expand Up @@ -205,8 +206,8 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA
// `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec`
withSQLConf(SQLConf.ORC_COMPRESSION.key -> "uncompressed") {
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "NONE")
val map1 = Map("orc.compress" -> "zlib")
val map2 = Map("orc.compress" -> "zlib", "compression" -> "lzo")
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")
}
Expand Down