-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24881][SQL] New Avro option - compression #21837
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 12 commits
311482a
0caa0f0
c5802df
f8b580b
c2373f2
8c1746c
d21d3e9
ac117a7
41f1936
952bdb0
b315f37
6915f34
5561582
ebaf327
5f83902
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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 { | ||
|
||
|
|
||
| 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) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -68,4 +70,11 @@ 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) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,8 +27,8 @@ 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 | ||
|
|
||
|
|
@@ -364,7 +364,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" | ||
|
||
| val AVRO_DEFLATE_LEVEL = "spark.sql.avro.deflate.level" | ||
|
||
|
|
@@ -904,4 +904,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 |
|---|---|---|
|
|
@@ -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 | ||
|
|
@@ -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.") | ||
|
||
| .stringConf | ||
| .checkValues(Set("uncompressed", "deflate", "snappy")) | ||
| .createWithDefault("snappy") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we |
||
|
|
||
| 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.") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we do the check like |
||
| .intConf | ||
| .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION) | ||
| .createWithDefault(Deflater.DEFAULT_COMPRESSION) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -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. */ | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Deflater.DEFAULT_COMPRESSIONis -1 here. Why change the default value to -6 in SQLConf?There was a problem hiding this comment.
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
-1means. Is it closer to best compression9or fast compression1? Probably the level is eventually set inzlibin which-1means6. @gengliangwang From your point of view,-1means better or faster compression?There was a problem hiding this comment.
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 useDeflater.DEFAULT_COMPRESSIONinstead of any specific number.