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 @@ -117,11 +117,9 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
dataSchema, nullable = false, parsedOptions.recordName, parsedOptions.recordNamespace)

AvroJob.setOutputKeySchema(job, outputAvroSchema)
val AVRO_COMPRESSION_CODEC = "spark.sql.avro.compression.codec"
val AVRO_DEFLATE_LEVEL = "spark.sql.avro.deflate.level"
val COMPRESS_KEY = "mapred.output.compress"

spark.conf.get(AVRO_COMPRESSION_CODEC, "snappy") match {
parsedOptions.compression match {
case "uncompressed" =>
log.info("writing uncompressed Avro records")
job.getConfiguration.setBoolean(COMPRESS_KEY, false)
Expand All @@ -132,8 +130,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, DataFileConstants.SNAPPY_CODEC)

case "deflate" =>
val deflateLevel = spark.conf.get(
AVRO_DEFLATE_LEVEL, Deflater.DEFAULT_COMPRESSION.toString).toInt
Copy link
Member

Choose a reason for hiding this comment

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

Deflater.DEFAULT_COMPRESSION is -1 here. Why change the default value to -6 in SQLConf?

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 changed it because I didn't have any ideas what -1 means. Is it closer to best compression 9 or fast compression 1? Probably the level is eventually set in zlib in which -1 means 6. @gengliangwang From your point of view, -1 means better or faster compression?

Copy link
Member

Choose a reason for hiding this comment

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

If the compression algorithm changes in the feature, the default value may not be 6. But if we use DEFAULT_COMPRESSION, it will still be equivalent to the new default value. So I suggest we still use Deflater.DEFAULT_COMPRESSION instead of any specific number.

val deflateLevel = spark.sessionState.conf.avroDeflateLevel
log.info(s"compressing Avro output using deflate (level=$deflateLevel)")
job.getConfiguration.setBoolean(COMPRESS_KEY, true)
job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, DataFileConstants.DEFLATE_CODEC)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration

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

/**
* Options for Avro Reader and Writer stored in case insensitive manner.
Expand Down Expand Up @@ -68,4 +69,14 @@ class AvroOptions(
.map(_.toBoolean)
.getOrElse(!ignoreFilesWithoutExtension)
}

/**
* The `compression` option allows to specify a compression codec used in write.
* Currently supported codecs are `uncompressed`, `snappy` and `deflate`.
* If the option is not set, the `spark.sql.avro.compression.codec` config is taken into
* account. If the former one is not set too, the `snappy` codec is used by default.
*/
val compression: String = {
parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,14 @@ import scala.collection.JavaConverters._

import org.apache.avro.Schema
import org.apache.avro.Schema.{Field, Type}
import org.apache.avro.file.DataFileWriter
import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord}
import org.apache.avro.file.{DataFileReader, DataFileWriter}
import org.apache.avro.generic.{GenericData, GenericDatumReader, GenericDatumWriter, GenericRecord}
import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed}
import org.apache.commons.io.FileUtils

import org.apache.spark.sql._
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
import org.apache.spark.sql.types._

Expand Down Expand Up @@ -364,21 +365,19 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
}
}

test("write with compression") {
test("write with compression - sql configs") {
withTempPath { dir =>
val AVRO_COMPRESSION_CODEC = "spark.sql.avro.compression.codec"
val AVRO_DEFLATE_LEVEL = "spark.sql.avro.deflate.level"
val uncompressDir = s"$dir/uncompress"
val deflateDir = s"$dir/deflate"
val snappyDir = s"$dir/snappy"

val df = spark.read.format("avro").load(testAvro)
spark.conf.set(AVRO_COMPRESSION_CODEC, "uncompressed")
spark.conf.set(SQLConf.AVRO_COMPRESSION_CODEC.key, "uncompressed")
df.write.format("avro").save(uncompressDir)
spark.conf.set(AVRO_COMPRESSION_CODEC, "deflate")
spark.conf.set(AVRO_DEFLATE_LEVEL, "9")
spark.conf.set(SQLConf.AVRO_COMPRESSION_CODEC.key, "deflate")
spark.conf.set(SQLConf.AVRO_DEFLATE_LEVEL.key, "9")
df.write.format("avro").save(deflateDir)
spark.conf.set(AVRO_COMPRESSION_CODEC, "snappy")
spark.conf.set(SQLConf.AVRO_COMPRESSION_CODEC.key, "snappy")
df.write.format("avro").save(snappyDir)

val uncompressSize = FileUtils.sizeOfDirectory(new File(uncompressDir))
Expand Down Expand Up @@ -904,4 +903,31 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
assert(count == 8)
}
}

test("SPARK-24881: write with compression - avro options") {
def getCodec(dir: String): Option[String] = {
val files = new File(dir)
.listFiles()
.filter(_.isFile)
.filter(_.getName.endsWith("avro"))
files.map { file =>
val reader = new DataFileReader(file, new GenericDatumReader[Any]())
val r = reader.getMetaString("avro.codec")
r
}.map(v => if (v == "null") "uncompressed" else v).headOption
}
def checkCodec(df: DataFrame, dir: String, codec: String): Unit = {
val subdir = s"$dir/$codec"
df.write.option("compression", codec).format("avro").save(subdir)
assert(getCodec(subdir) == Some(codec))
}
withTempPath { dir =>
val path = dir.toString
val df = spark.read.format("avro").load(testAvro)

checkCodec(df, path, "uncompressed")
checkCodec(df, path, "deflate")
checkCodec(df, path, "snappy")
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.internal
import java.util.{Locale, NoSuchElementException, Properties, TimeZone}
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicReference
import java.util.zip.Deflater

import scala.collection.JavaConverters._
import scala.collection.immutable
Expand Down Expand Up @@ -1434,6 +1435,20 @@ object SQLConf {
"This only takes effect when spark.sql.repl.eagerEval.enabled is set to true.")
.intConf
.createWithDefault(20)

val AVRO_COMPRESSION_CODEC = buildConf("spark.sql.avro.compression.codec")
.doc("Compression codec used in writing of AVRO files. Default codec is snappy.")
.stringConf
.checkValues(Set("uncompressed", "deflate", "snappy"))
.createWithDefault("snappy")

val AVRO_DEFLATE_LEVEL = buildConf("spark.sql.avro.deflate.level")
.doc("Compression level for the deflate codec used in writing of AVRO files. " +
"Valid value must be in the range of from 1 to 9 inclusive or -1. " +
"The default value is -1 which corresponds to 6 level in the current implementation.")
.intConf
.checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION)
.createWithDefault(Deflater.DEFAULT_COMPRESSION)
}

/**
Expand Down Expand Up @@ -1820,6 +1835,10 @@ class SQLConf extends Serializable with Logging {

def replEagerEvalTruncate: Int = getConf(SQLConf.REPL_EAGER_EVAL_TRUNCATE)

def avroCompressionCodec: String = getConf(SQLConf.AVRO_COMPRESSION_CODEC)

def avroDeflateLevel: Int = getConf(SQLConf.AVRO_DEFLATE_LEVEL)

/** ********************** SQLConf functionality methods ************ */

/** Set Spark SQL configuration properties. */
Expand Down