Skip to content

Commit 561505e

Browse files
dongjoon-hyungatorsmile
authored andcommitted
[SPARK-22282][SQL] Rename OrcRelation to OrcFileFormat and remove ORC_COMPRESSION
## What changes were proposed in this pull request? This PR aims to - Rename `OrcRelation` to `OrcFileFormat` object. - Replace `OrcRelation.ORC_COMPRESSION` with `org.apache.orc.OrcConf.COMPRESS`. Since [SPARK-21422](https://issues.apache.org/jira/browse/SPARK-21422), we can use `OrcConf.COMPRESS` instead of Hive's. ```scala // The references of Hive's classes will be minimized. val ORC_COMPRESSION = "orc.compress" ``` ## How was this patch tested? Pass the Jenkins with the existing and updated test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19502 from dongjoon-hyun/SPARK-22282.
1 parent 0fa1066 commit 561505e

File tree

5 files changed

+27
-23
lines changed

5 files changed

+27
-23
lines changed

sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -520,8 +520,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) {
520520
* <li>`compression` (default is the value specified in `spark.sql.orc.compression.codec`):
521521
* compression codec to use when saving to file. This can be one of the known case-insensitive
522522
* shorten names(`none`, `snappy`, `zlib`, and `lzo`). This will override
523-
* `orc.compress` and `spark.sql.parquet.compression.codec`. If `orc.compress` is given,
524-
* it overrides `spark.sql.parquet.compression.codec`.</li>
523+
* `orc.compress` and `spark.sql.orc.compression.codec`. If `orc.compress` is given,
524+
* it overrides `spark.sql.orc.compression.codec`.</li>
525525
* </ul>
526526
*
527527
* @since 1.5.0

sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala

Lines changed: 8 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ import org.apache.hadoop.io.{NullWritable, Writable}
3232
import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter}
3333
import org.apache.hadoop.mapreduce._
3434
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
35+
import org.apache.orc.OrcConf.COMPRESS
3536

3637
import org.apache.spark.TaskContext
3738
import org.apache.spark.sql.SparkSession
@@ -72,7 +73,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable
7273

7374
val configuration = job.getConfiguration
7475

75-
configuration.set(OrcRelation.ORC_COMPRESSION, orcOptions.compressionCodec)
76+
configuration.set(COMPRESS.getAttribute, orcOptions.compressionCodec)
7677
configuration match {
7778
case conf: JobConf =>
7879
conf.setOutputFormat(classOf[OrcOutputFormat])
@@ -93,8 +94,8 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable
9394

9495
override def getFileExtension(context: TaskAttemptContext): String = {
9596
val compressionExtension: String = {
96-
val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION)
97-
OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "")
97+
val name = context.getConfiguration.get(COMPRESS.getAttribute)
98+
OrcFileFormat.extensionsForCompressionCodecNames.getOrElse(name, "")
9899
}
99100

100101
compressionExtension + ".orc"
@@ -120,7 +121,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable
120121
if (sparkSession.sessionState.conf.orcFilterPushDown) {
121122
// Sets pushed predicates
122123
OrcFilters.createFilter(requiredSchema, filters.toArray).foreach { f =>
123-
hadoopConf.set(OrcRelation.SARG_PUSHDOWN, f.toKryo)
124+
hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, f.toKryo)
124125
hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true)
125126
}
126127
}
@@ -138,7 +139,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable
138139
if (isEmptyFile) {
139140
Iterator.empty
140141
} else {
141-
OrcRelation.setRequiredColumns(conf, dataSchema, requiredSchema)
142+
OrcFileFormat.setRequiredColumns(conf, dataSchema, requiredSchema)
142143

143144
val orcRecordReader = {
144145
val job = Job.getInstance(conf)
@@ -160,7 +161,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable
160161
Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => recordsIterator.close()))
161162

162163
// Unwraps `OrcStruct`s to `UnsafeRow`s
163-
OrcRelation.unwrapOrcStructs(
164+
OrcFileFormat.unwrapOrcStructs(
164165
conf,
165166
dataSchema,
166167
requiredSchema,
@@ -255,10 +256,7 @@ private[orc] class OrcOutputWriter(
255256
}
256257
}
257258

258-
private[orc] object OrcRelation extends HiveInspectors {
259-
// The references of Hive's classes will be minimized.
260-
val ORC_COMPRESSION = "orc.compress"
261-
259+
private[orc] object OrcFileFormat extends HiveInspectors {
262260
// This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public.
263261
private[orc] val SARG_PUSHDOWN = "sarg.pushdown"
264262

sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.orc
1919

2020
import java.util.Locale
2121

22+
import org.apache.orc.OrcConf.COMPRESS
23+
2224
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
2325
import org.apache.spark.sql.internal.SQLConf
2426

@@ -40,9 +42,9 @@ private[orc] class OrcOptions(
4042
* Acceptable values are defined in [[shortOrcCompressionCodecNames]].
4143
*/
4244
val compressionCodec: String = {
43-
// `compression`, `orc.compress`, and `spark.sql.orc.compression.codec` are
44-
// in order of precedence from highest to lowest.
45-
val orcCompressionConf = parameters.get(OrcRelation.ORC_COMPRESSION)
45+
// `compression`, `orc.compress`(i.e., OrcConf.COMPRESS), and `spark.sql.orc.compression.codec`
46+
// are in order of precedence from highest to lowest.
47+
val orcCompressionConf = parameters.get(COMPRESS.getAttribute)
4648
val codecName = parameters
4749
.get("compression")
4850
.orElse(orcCompressionConf)

sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcQuerySuite.scala

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.sql.Timestamp
2222

2323
import org.apache.hadoop.conf.Configuration
2424
import org.apache.hadoop.hive.ql.io.orc.{OrcStruct, SparkOrcNewRecordReader}
25+
import org.apache.orc.OrcConf.COMPRESS
2526
import org.scalatest.BeforeAndAfterAll
2627

2728
import org.apache.spark.sql._
@@ -176,11 +177,11 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
176177
}
177178
}
178179

179-
test("SPARK-16610: Respect orc.compress option when compression is unset") {
180-
// Respect `orc.compress`.
180+
test("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") {
181+
// Respect `orc.compress` (i.e., OrcConf.COMPRESS).
181182
withTempPath { file =>
182183
spark.range(0, 10).write
183-
.option("orc.compress", "ZLIB")
184+
.option(COMPRESS.getAttribute, "ZLIB")
184185
.orc(file.getCanonicalPath)
185186
val expectedCompressionKind =
186187
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
@@ -191,7 +192,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
191192
withTempPath { file =>
192193
spark.range(0, 10).write
193194
.option("compression", "ZLIB")
194-
.option("orc.compress", "SNAPPY")
195+
.option(COMPRESS.getAttribute, "SNAPPY")
195196
.orc(file.getCanonicalPath)
196197
val expectedCompressionKind =
197198
OrcFileOperator.getFileReader(file.getCanonicalPath).get.getCompression
@@ -598,7 +599,7 @@ class OrcQuerySuite extends QueryTest with BeforeAndAfterAll with OrcTest {
598599
val requestedSchema = StructType(Nil)
599600
val conf = new Configuration()
600601
val physicalSchema = OrcFileOperator.readSchema(Seq(path), Some(conf)).get
601-
OrcRelation.setRequiredColumns(conf, physicalSchema, requestedSchema)
602+
OrcFileFormat.setRequiredColumns(conf, physicalSchema, requestedSchema)
602603
val maybeOrcReader = OrcFileOperator.getFileReader(path, Some(conf))
603604
assert(maybeOrcReader.isDefined)
604605
val orcRecordReader = new SparkOrcNewRecordReader(

sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcSourceSuite.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,9 @@
1818
package org.apache.spark.sql.hive.orc
1919

2020
import java.io.File
21+
import java.util.Locale
2122

23+
import org.apache.orc.OrcConf.COMPRESS
2224
import org.scalatest.BeforeAndAfterAll
2325

2426
import org.apache.spark.sql.{QueryTest, Row}
@@ -150,7 +152,8 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA
150152

151153
test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
152154
val conf = sqlContext.sessionState.conf
153-
assert(new OrcOptions(Map("Orc.Compress" -> "NONE"), conf).compressionCodec == "NONE")
155+
val option = new OrcOptions(Map(COMPRESS.getAttribute.toUpperCase(Locale.ROOT) -> "NONE"), conf)
156+
assert(option.compressionCodec == "NONE")
154157
}
155158

156159
test("SPARK-19459/SPARK-18220: read char/varchar column written by Hive") {
@@ -205,8 +208,8 @@ abstract class OrcSuite extends QueryTest with TestHiveSingleton with BeforeAndA
205208
// `compression` -> `orc.compression` -> `spark.sql.orc.compression.codec`
206209
withSQLConf(SQLConf.ORC_COMPRESSION.key -> "uncompressed") {
207210
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == "NONE")
208-
val map1 = Map("orc.compress" -> "zlib")
209-
val map2 = Map("orc.compress" -> "zlib", "compression" -> "lzo")
211+
val map1 = Map(COMPRESS.getAttribute -> "zlib")
212+
val map2 = Map(COMPRESS.getAttribute -> "zlib", "compression" -> "lzo")
210213
assert(new OrcOptions(map1, conf).compressionCodec == "ZLIB")
211214
assert(new OrcOptions(map2, conf).compressionCodec == "LZO")
212215
}

0 commit comments

Comments
 (0)