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
@@ -0,0 +1,56 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.datasources.orc;

import java.util.Arrays;
import java.util.Locale;
import java.util.Map;
import java.util.stream.Collectors;

import org.apache.orc.CompressionKind;

/**
* A mapper class from Spark supported orc compression codecs to orc compression codecs.
*/
public enum OrcCompressionCodec {
NONE(CompressionKind.NONE),
UNCOMPRESSED(CompressionKind.NONE),
ZLIB(CompressionKind.ZLIB),
SNAPPY(CompressionKind.SNAPPY),
LZO(CompressionKind.LZO),
LZ4(CompressionKind.LZ4),
ZSTD(CompressionKind.ZSTD);

private final CompressionKind compressionKind;

OrcCompressionCodec(CompressionKind compressionKind) {
this.compressionKind = compressionKind;
}

public CompressionKind getCompressionKind() {
return this.compressionKind;
}

public static final Map<String, String> codecNameMap =
Arrays.stream(OrcCompressionCodec.values()).collect(
Collectors.toMap(codec -> codec.name(), codec -> codec.name().toLowerCase(Locale.ROOT)));

public String lowerCaseName() {
return codecNameMap.get(this.name());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -75,14 +75,9 @@ object OrcOptions extends DataSourceOptions {
val COMPRESSION = newOption("compression")

// The ORC compression short names
private val shortOrcCompressionCodecNames = Map(
"none" -> "NONE",
"uncompressed" -> "NONE",
"snappy" -> "SNAPPY",
"zlib" -> "ZLIB",
"lzo" -> "LZO",
"lz4" -> "LZ4",
"zstd" -> "ZSTD")
private val shortOrcCompressionCodecNames = OrcCompressionCodec.values().map {
mapper => mapper.lowerCaseName() -> mapper.getCompressionKind.name()
}.toMap

def getORCCompressionCodecName(name: String): String = shortOrcCompressionCodecNames(name)
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,12 +49,12 @@ object OrcUtils extends Logging {

// The extensions for ORC compression codecs
val extensionsForCompressionCodecNames = Map(
"NONE" -> "",
"SNAPPY" -> ".snappy",
"ZLIB" -> ".zlib",
"ZSTD" -> ".zstd",
"LZ4" -> ".lz4",
"LZO" -> ".lzo")
OrcCompressionCodec.NONE.name() -> "",
OrcCompressionCodec.SNAPPY.name() -> ".snappy",
OrcCompressionCodec.ZLIB.name() -> ".zlib",
OrcCompressionCodec.ZSTD.name() -> ".zstd",
OrcCompressionCodec.LZ4.name() -> ".lz4",
OrcCompressionCodec.LZO.name() -> ".lzo")

val CATALYST_TYPE_ATTRIBUTE_NAME = "spark.sql.catalyst.type"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import java.util.Locale
import org.apache.parquet.column.ParquetProperties
import org.apache.parquet.hadoop.ParquetOutputFormat

import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec
import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec
import org.apache.spark.sql.internal.SQLConf

Expand Down Expand Up @@ -56,7 +57,8 @@ object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark {

spark.conf.set(SQLConf.PARQUET_COMPRESSION.key,
ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT))
spark.conf.set(SQLConf.ORC_COMPRESSION.key, "snappy")
spark.conf.set(SQLConf.ORC_COMPRESSION.key,
OrcCompressionCodec.SNAPPY.lowerCaseName())

formats.foreach { format =>
runBenchmark(s"$format writer benchmark") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import org.apache.spark.{SparkConf, TestUtils}
import org.apache.spark.benchmark.Benchmark
import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec
import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, VectorizedParquetRecordReader}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
Expand Down Expand Up @@ -116,7 +117,8 @@ object DataSourceReadBenchmark extends SqlBasedBenchmark {
}

private def saveAsOrcTable(df: DataFrameWriter[Row], dir: String): Unit = {
df.mode("overwrite").option("compression", "snappy").orc(dir)
df.mode("overwrite").option("compression",
OrcCompressionCodec.SNAPPY.lowerCaseName()).orc(dir)
spark.read.orc(dir).createOrReplaceTempView("orcTable")
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import scala.util.Random
import org.apache.spark.SparkConf
import org.apache.spark.benchmark.Benchmark
import org.apache.spark.sql.{DataFrame, SparkSession}
import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec
import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec
import org.apache.spark.sql.functions.{monotonically_increasing_id, timestamp_seconds}
import org.apache.spark.sql.internal.SQLConf
Expand All @@ -51,7 +52,7 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark {
.set("spark.master", "local[1]")
.setIfMissing("spark.driver.memory", "3g")
.setIfMissing("spark.executor.memory", "3g")
.setIfMissing("orc.compression", "snappy")
.setIfMissing("orc.compression", OrcCompressionCodec.SNAPPY.lowerCaseName())
.setIfMissing("spark.sql.parquet.compression.codec",
ParquetCompressionCodec.SNAPPY.name().toLowerCase(Locale.ROOT))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import java.util.Locale
import scala.jdk.CollectionConverters._

import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.execution.datasources.orc.OrcCompressionCodec
import org.apache.spark.sql.execution.datasources.parquet.ParquetCompressionCodec
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
Expand Down Expand Up @@ -73,6 +74,6 @@ class OrcCodecSuite extends FileSourceCodecSuite {

override def format: String = "orc"
override val codecConfigName: String = SQLConf.ORC_COMPRESSION.key
override protected def availableCodecs = Seq("none", "uncompressed", "snappy",
"zlib", "zstd", "lz4", "lzo")
override protected def availableCodecs =
OrcCompressionCodec.values().map(_.lowerCaseName()).toSeq
}
Original file line number Diff line number Diff line change
Expand Up @@ -190,7 +190,7 @@ abstract class OrcQueryTest extends OrcTest {
// Respect `orc.compress` (i.e., OrcConf.COMPRESS).
withTempPath { file =>
spark.range(0, 10).write
.option(COMPRESS.getAttribute, "ZLIB")
.option(COMPRESS.getAttribute, OrcCompressionCodec.ZLIB.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".zlib.orc"))
Expand All @@ -199,15 +199,15 @@ abstract class OrcQueryTest extends OrcTest {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("ZLIB" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.ZLIB.name() === reader.getCompressionKind.name)
}
}

// `compression` overrides `orc.compress`.
withTempPath { file =>
spark.range(0, 10).write
.option("compression", "ZLIB")
.option(COMPRESS.getAttribute, "SNAPPY")
.option("compression", OrcCompressionCodec.ZLIB.name())
.option(COMPRESS.getAttribute, OrcCompressionCodec.SNAPPY.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".zlib.orc"))
Expand All @@ -216,15 +216,15 @@ abstract class OrcQueryTest extends OrcTest {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("ZLIB" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.ZLIB.name() === reader.getCompressionKind.name)
}
}
}

test("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") {
withTempPath { file =>
spark.range(0, 10).write
.option("compression", "ZLIB")
.option("compression", OrcCompressionCodec.ZLIB.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".zlib.orc"))
Expand All @@ -233,13 +233,13 @@ abstract class OrcQueryTest extends OrcTest {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("ZLIB" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.ZLIB.name() === reader.getCompressionKind.name)
}
}

withTempPath { file =>
spark.range(0, 10).write
.option("compression", "SNAPPY")
.option("compression", OrcCompressionCodec.SNAPPY.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".snappy.orc"))
Expand All @@ -248,13 +248,13 @@ abstract class OrcQueryTest extends OrcTest {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("SNAPPY" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.SNAPPY.name() === reader.getCompressionKind.name)
}
}

withTempPath { file =>
spark.range(0, 10).write
.option("compression", "NONE")
.option("compression", OrcCompressionCodec.NONE.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".orc"))
Expand All @@ -263,7 +263,7 @@ abstract class OrcQueryTest extends OrcTest {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("NONE" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.NONE.name() === reader.getCompressionKind.name)
}
}
}
Expand Down Expand Up @@ -647,7 +647,7 @@ abstract class OrcQuerySuite extends OrcQueryTest with SharedSparkSession {
test("LZO compression options for writing to an ORC file") {
withTempPath { file =>
spark.range(0, 10).write
.option("compression", "LZO")
.option("compression", OrcCompressionCodec.LZO.name())
.orc(file.getCanonicalPath)

val maybeOrcFile = file.listFiles().find(_.getName.endsWith(".lzo.orc"))
Expand All @@ -656,7 +656,7 @@ abstract class OrcQuerySuite extends OrcQueryTest with SharedSparkSession {
val orcFilePath = new Path(maybeOrcFile.get.getAbsolutePath)
val conf = OrcFile.readerOptions(new Configuration())
Utils.tryWithResource(OrcFile.createReader(orcFilePath, conf)) { reader =>
assert("LZO" === reader.getCompressionKind.name)
assert(OrcCompressionCodec.LZO.name() === reader.getCompressionKind.name)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -324,29 +324,38 @@ abstract class OrcSuite

test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") {
val conf = spark.sessionState.conf
val option = new OrcOptions(Map(COMPRESS.getAttribute.toUpperCase(Locale.ROOT) -> "NONE"), conf)
assert(option.compressionCodec == "NONE")
val option = new OrcOptions(
Map(COMPRESS.getAttribute.toUpperCase(Locale.ROOT) -> OrcCompressionCodec.NONE.name()), conf)
assert(option.compressionCodec == OrcCompressionCodec.NONE.name())
}

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")
assert(new OrcOptions(
Map.empty[String, String], conf).compressionCodec == OrcCompressionCodec.SNAPPY.name())

// OrcOptions's parameters have a higher priority than SQL configuration.
// `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(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")
assert(new OrcOptions(
Map.empty[String, String], conf).compressionCodec == OrcCompressionCodec.NONE.name())
val zlibCodec = OrcCompressionCodec.ZLIB.lowerCaseName()
val lzoCodec = OrcCompressionCodec.LZO.lowerCaseName()
val map1 = Map(COMPRESS.getAttribute -> zlibCodec)
val map2 = Map(COMPRESS.getAttribute -> zlibCodec, "compression" -> lzoCodec)
assert(new OrcOptions(map1, conf).compressionCodec == OrcCompressionCodec.ZLIB.name())
assert(new OrcOptions(map2, conf).compressionCodec == OrcCompressionCodec.LZO.name())
}

// Test all the valid options of spark.sql.orc.compression.codec
Seq("NONE", "UNCOMPRESSED", "SNAPPY", "ZLIB", "LZO", "ZSTD", "LZ4").foreach { c =>
OrcCompressionCodec.values().map(_.name()).foreach { c =>
withSQLConf(SQLConf.ORC_COMPRESSION.key -> c) {
val expected = if (c == "UNCOMPRESSED") "NONE" else c
val expected = if (c == OrcCompressionCodec.UNCOMPRESSED.name()) {
OrcCompressionCodec.NONE.name()
} else {
c
}
assert(new OrcOptions(Map.empty[String, String], conf).compressionCodec == expected)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,11 @@ import java.util.Locale
import scala.jdk.CollectionConverters._

import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.ql.io.orc.CompressionKind
import org.apache.orc.OrcConf.COMPRESS
import org.apache.parquet.hadoop.ParquetOutputFormat
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.sql.execution.datasources.orc.OrcOptions
import org.apache.spark.sql.execution.datasources.orc.{OrcCompressionCodec, OrcOptions}
import org.apache.spark.sql.execution.datasources.parquet.{ParquetCompressionCodec, ParquetOptions, ParquetTest}
import org.apache.spark.sql.hive.orc.OrcFileOperator
import org.apache.spark.sql.hive.test.TestHiveSingleton
Expand Down Expand Up @@ -299,9 +298,15 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo
ParquetCompressionCodec.SNAPPY.name))
checkForTableWithCompressProp("orc",
tableCompressCodecs =
List(CompressionKind.NONE.name, CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name),
List(
OrcCompressionCodec.NONE.name,
OrcCompressionCodec.SNAPPY.name,
OrcCompressionCodec.ZLIB.name),
sessionCompressCodecs =
List(CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name, CompressionKind.SNAPPY.name))
List(
OrcCompressionCodec.SNAPPY.name,
OrcCompressionCodec.ZLIB.name,
OrcCompressionCodec.SNAPPY.name))
}

test("table-level compression is not set but session-level compressions is set ") {
Expand All @@ -314,7 +319,10 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo
checkForTableWithCompressProp("orc",
tableCompressCodecs = List.empty,
sessionCompressCodecs =
List(CompressionKind.NONE.name, CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name))
List(
OrcCompressionCodec.NONE.name,
OrcCompressionCodec.SNAPPY.name,
OrcCompressionCodec.ZLIB.name))
}

def checkTableWriteWithCompressionCodecs(format: String, compressCodecs: List[String]): Unit = {
Expand Down Expand Up @@ -355,6 +363,9 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo
ParquetCompressionCodec.GZIP.name))
checkTableWriteWithCompressionCodecs(
"orc",
List(CompressionKind.NONE.name, CompressionKind.SNAPPY.name, CompressionKind.ZLIB.name))
List(
OrcCompressionCodec.NONE.name,
OrcCompressionCodec.SNAPPY.name,
OrcCompressionCodec.ZLIB.name))
}
}
Loading