Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 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 @@ -58,7 +58,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
options: Map[String, String],
files: Seq[FileStatus]): Option[StructType] = {
val conf = spark.sparkContext.hadoopConfiguration
val parsedOptions = new AvroOptions(options, conf)
val parsedOptions = new AvroOptions(options, conf, spark.sessionState.conf)

// Schema evolution is not supported yet. Here we only pick a single random sample file to
// figure out the schema of the whole dataset.
Expand Down Expand Up @@ -113,16 +113,17 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
val parsedOptions = new AvroOptions(options, spark.sessionState.newHadoopConf())
val parsedOptions = new AvroOptions(
options,
spark.sessionState.newHadoopConf(),
spark.sessionState.conf)
val outputAvroSchema = SchemaConverters.toAvroType(
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 @@ -133,8 +134,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 = parsedOptions.compressionLevel
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 All @@ -158,7 +158,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister {

val broadcastedConf =
spark.sparkContext.broadcast(new AvroFileFormat.SerializableConfiguration(hadoopConf))
val parsedOptions = new AvroOptions(options, hadoopConf)
val parsedOptions = new AvroOptions(options, hadoopConf, spark.sessionState.conf)

(file: PartitionedFile) => {
val log = LoggerFactory.getLogger(classOf[AvroFileFormat])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,18 @@ 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.
*/
class AvroOptions(
@transient val parameters: CaseInsensitiveMap[String],
@transient val conf: Configuration) extends Logging with Serializable {
@transient val conf: Configuration,
@transient val sqlConf: SQLConf) extends Logging with Serializable {
Copy link
Member

Choose a reason for hiding this comment

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

We may just get SQLConf by calling SQLConf.get without passing it in.


def this(parameters: Map[String, String], conf: Configuration) = {
this(CaseInsensitiveMap(parameters), conf)
def this(parameters: Map[String, String], conf: Configuration, sqlConf: SQLConf) = {
this(CaseInsensitiveMap(parameters), conf, sqlConf)
}

/**
Expand Down Expand Up @@ -68,4 +70,25 @@ 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 `snappy` compression is used by default.
*/
val compression: String = parameters.get("compression").getOrElse(sqlConf.avroCompressionCodec)


/**
* Level of compression in the range of 1..9 inclusive. 1 - for fast, 9 - for best compression.
* If the compression level is not set for `deflate` compression, the current value of SQL
* config `spark.sql.avro.deflate.level` is used by default. For other compressions, the default
* value is `6`.
*/
val compressionLevel: Int = {
Copy link
Member

@HyukjinKwon HyukjinKwon Jul 24, 2018

Choose a reason for hiding this comment

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

Can we don't expose this as an option for now? IIUC, this compression level only applies to deflate, right? Also, this option looks not for keeping the same options from the thrid party as well.

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 added the option keeping in mind other compression codecs can be added in the future, for example zstandard. For those codecs, the level could be useful too. Another point is specifying compression level together with compression codec in Avro options looks more natural comparing to SQL global settings:

df.write
  .options(Map("compression" -> "deflate", "compressionLevel" -> "9"))
  .format("avro")
  .save(deflateDir)

vs

spark.conf.set("spark.sql.avro.deflate.level", "9")
df.write
  .option("compression", "deflate"))
  .format("avro")
  .save(deflateDir)

Copy link
Member

Choose a reason for hiding this comment

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

Yea, I know that could be useful in some ways but I was thinking we should better not add this just for now. Thing is, it sounds currently too specific to one compression option in Avro for now .. There are many options to expose in, for example in CSV datasource too in this way ..

Copy link
Member

Choose a reason for hiding this comment

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

Also, to be honest, I wonder users would want to change compression level often ..

val defaultLevel = 6
parameters.get("compressionLevel").map(_.toInt).getOrElse {
if (compression == "deflate") sqlConf.avroDeflateLevel else defaultLevel
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -355,7 +355,7 @@ 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"
Copy link
Member

Choose a reason for hiding this comment

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

This can use SQLConf.AVRO_COMPRESSION_CODEC.key now.

val AVRO_DEFLATE_LEVEL = "spark.sql.avro.deflate.level"
Copy link
Member

Choose a reason for hiding this comment

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

ditto.

Expand Down Expand Up @@ -889,4 +889,30 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
assert(count == 8)
}
}

test("SPARK-24881: write with compression - avro options") {
withTempPath { dir =>
val uncompressDir = s"$dir/uncompress"
val deflateDir = s"$dir/deflate"
val snappyDir = s"$dir/snappy"

val df = spark.read.avro(testAvro)
Copy link
Member

Choose a reason for hiding this comment

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

I am removing all the .avro method: #21841
Could you change it to ".format("avro").load" or ".format("avro").save"? Thanks!

df.write
.option("compression", "uncompressed")
.avro(uncompressDir)
df.write
.options(Map("compression" -> "deflate", "compressionLevel" -> "9"))
.avro(deflateDir)
df.write
.option("compression", "snappy")
.avro(snappyDir)

val uncompressSize = FileUtils.sizeOfDirectory(new File(uncompressDir))
val deflateSize = FileUtils.sizeOfDirectory(new File(deflateDir))
Copy link
Member

Choose a reason for hiding this comment

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

is there any or easy way to check the metadata for compression level?

Copy link
Member

Choose a reason for hiding this comment

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

Thank you, @MaxGekk. Can we then check the type of compression at least avro.codec deflate?

val snappySize = FileUtils.sizeOfDirectory(new File(snappyDir))

assert(uncompressSize > deflateSize)
assert(snappySize > deflateSize)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1422,6 +1422,17 @@ 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.")
Copy link
Member

Choose a reason for hiding this comment

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

Document the default value?

.stringConf
.createWithDefault("snappy")
Copy link
Member

Choose a reason for hiding this comment

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

can we .checkValues(Set( too?


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. Default value is 6.")
.intConf
.createWithDefault(6)
}

/**
Expand Down Expand Up @@ -1806,6 +1817,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