From cdb26f27e83c5340657c967b0b1bc72373e8e606 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Jul 2018 17:18:33 +0200 Subject: [PATCH 1/7] Adding new option - ignoreExtension --- .../spark/sql/avro/AvroFileFormat.scala | 16 ++++++----- .../org/apache/spark/sql/avro/AvroSuite.scala | 27 +++++++------------ 2 files changed, 20 insertions(+), 23 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 9eb206457809c..3aaab9a273427 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -62,7 +62,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Schema evolution is not supported yet. Here we only pick a single random sample file to // figure out the schema of the whole dataset. val sampleFile = - if (AvroFileFormat.ignoreFilesWithoutExtensions(conf)) { + if (AvroFileFormat.ignoreFilesWithoutExtensions(conf, options)) { files.find(_.getPath.getName.endsWith(".avro")).getOrElse { throw new FileNotFoundException( "No Avro files found. Hadoop option \"avro.mapred.ignore.inputs.without.extension\" " + @@ -170,7 +170,8 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. - if (AvroFileFormat.ignoreFilesWithoutExtensions(conf) && !file.filePath.endsWith(".avro")) { + if (AvroFileFormat.ignoreFilesWithoutExtensions(conf, options) && + !file.filePath.endsWith(".avro")) { Iterator.empty } else { val reader = { @@ -276,10 +277,13 @@ private[avro] object AvroFileFormat { } } - def ignoreFilesWithoutExtensions(conf: Configuration): Boolean = { - // Files without .avro extensions are not ignored by default - val defaultValue = false + def ignoreFilesWithoutExtensions(conf: Configuration, options: Map[String, String]): Boolean = { + val ignoreExtensionByDefault = true + val ignoreExtension = options + .get("ignoreExtension") + .map(_.toBoolean) + .getOrElse(ignoreExtensionByDefault) - conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, defaultValue) + conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, !ignoreExtension) } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 446b42124ceca..b0b9ae4c98e02 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -623,12 +623,11 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { intercept[FileNotFoundException] { withTempPath { dir => FileUtils.touch(new File(dir, "test")) - val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration - try { - hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") - spark.read.avro(dir.toString) - } finally { - hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) } + + spark + .read + .option("ignoreExtension", false) + .avro(dir.toString) } } @@ -690,18 +689,12 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Files.createFile(new File(tempSaveDir, "non-avro").toPath) - val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration - val count = try { - hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") - val newDf = spark - .read - .avro(tempSaveDir) - newDf.count() - } finally { - hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) - } + val newDf = spark + .read + .option("ignoreExtension", false) + .avro(tempSaveDir) - assert(count == 8) + assert(newDf.count == 8) } } From d07ced332ba04ac18f270962d887f1e19e9155ec Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Jul 2018 17:23:32 +0200 Subject: [PATCH 2/7] Revert tests for hadoop's config --- .../org/apache/spark/sql/avro/AvroSuite.scala | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index b0b9ae4c98e02..ee22b88054e4f 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -620,6 +620,19 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { spark.read.avro("*/*/*/*/*/*/*/something.avro") } + intercept[FileNotFoundException] { + withTempPath { dir => + FileUtils.touch(new File(dir, "test")) + val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration + try { + hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") + spark.read.avro(dir.toString) + } finally { + hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) + } + } + } + intercept[FileNotFoundException] { withTempPath { dir => FileUtils.touch(new File(dir, "test")) @@ -630,7 +643,6 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { .avro(dir.toString) } } - } test("SQL test insert overwrite") { @@ -689,6 +701,18 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Files.createFile(new File(tempSaveDir, "non-avro").toPath) + val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration + val count = try { + hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") + val newDf = spark + .read + .avro(tempSaveDir) + newDf.count() + } finally { + hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) + } + assert(count == 8) + val newDf = spark .read .option("ignoreExtension", false) From e86d231ecaef45b816917517482732ab010ec9d8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Jul 2018 21:10:06 +0200 Subject: [PATCH 3/7] Adding the ignoreExtension option --- .../spark/sql/avro/AvroFileFormat.scala | 35 +++++++------- .../org/apache/spark/sql/avro/AvroSuite.scala | 48 ++++++++++++++++--- 2 files changed, 58 insertions(+), 25 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 3aaab9a273427..a7d299b5113a7 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -62,16 +62,14 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Schema evolution is not supported yet. Here we only pick a single random sample file to // figure out the schema of the whole dataset. val sampleFile = - if (AvroFileFormat.ignoreFilesWithoutExtensions(conf, options)) { - files.find(_.getPath.getName.endsWith(".avro")).getOrElse { - throw new FileNotFoundException( - "No Avro files found. Hadoop option \"avro.mapred.ignore.inputs.without.extension\" " + - " is set to true. Do all input files have \".avro\" extension?" - ) + if (AvroFileFormat.ignoreExtension(conf, options)) { + files.headOption.getOrElse { + throw new FileNotFoundException("Files for schema inferring have been not found.") } } else { - files.headOption.getOrElse { - throw new FileNotFoundException("No Avro files found.") + files.find(_.getPath.getName.endsWith(".avro")).getOrElse { + throw new FileNotFoundException( + "No Avro files found. If files don't have .avro extension, set ignoreExtension to true") } } @@ -170,10 +168,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. - if (AvroFileFormat.ignoreFilesWithoutExtensions(conf, options) && - !file.filePath.endsWith(".avro")) { - Iterator.empty - } else { + if (AvroFileFormat.ignoreExtension(conf, options) || file.filePath.endsWith(".avro")) { val reader = { val in = new FsInput(new Path(new URI(file.filePath)), conf) try { @@ -228,6 +223,8 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { deserializer.deserialize(record).asInstanceOf[InternalRow] } } + } else { + Iterator.empty } } } @@ -277,13 +274,15 @@ private[avro] object AvroFileFormat { } } - def ignoreFilesWithoutExtensions(conf: Configuration, options: Map[String, String]): Boolean = { - val ignoreExtensionByDefault = true - val ignoreExtension = options + def ignoreExtension(conf: Configuration, options: Map[String, String]): Boolean = { + val ignoreFilesWithoutExtensionByDefault = false + val ignoreFilesWithoutExtension = conf.getBoolean( + AvroFileFormat.IgnoreFilesWithoutExtensionProperty, + ignoreFilesWithoutExtensionByDefault) + + options .get("ignoreExtension") .map(_.toBoolean) - .getOrElse(ignoreExtensionByDefault) - - conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, !ignoreExtension) + .getOrElse(!ignoreFilesWithoutExtension) } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index ee22b88054e4f..867da19d8f9a7 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -712,13 +712,6 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) } assert(count == 8) - - val newDf = spark - .read - .option("ignoreExtension", false) - .avro(tempSaveDir) - - assert(newDf.count == 8) } } @@ -853,4 +846,45 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { assert(df2.count == 8) } } + + test("checking the ignoreExtension option") { + withTempPath { tempDir => + val df = spark.read.avro(episodesAvro) + assert(df.count == 8) + + val tempSaveDir = s"$tempDir/save/" + df.write.avro(tempSaveDir) + + Files.createFile(new File(tempSaveDir, "non-avro").toPath) + + val newDf = spark + .read + .option("ignoreExtension", false) + .avro(tempSaveDir) + + assert(newDf.count == 8) + } + } + + test("ignoreExtension must override hadoop's config") { + withTempDir { dir => + Files.copy( + Paths.get(new URL(episodesAvro).toURI), + Paths.get(dir.getCanonicalPath, "episodes")) + + val hadoopConf = spark.sqlContext.sparkContext.hadoopConfiguration + val count = try { + hadoopConf.set(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, "true") + val newDf = spark + .read + .option("ignoreExtension", "true") + .avro(s"${dir.getCanonicalPath}/episodes") + newDf.count() + } finally { + hadoopConf.unset(AvroFileFormat.IgnoreFilesWithoutExtensionProperty) + } + + assert(count == 8) + } + } } From 565e599cebceaec706ecd1c3c80a4e0c31ce3658 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Jul 2018 21:17:19 +0200 Subject: [PATCH 4/7] Ticket's number to tests titles is added --- .../src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 867da19d8f9a7..f8af2dd3f52ee 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -847,7 +847,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } - test("checking the ignoreExtension option") { + test("SPARK-24836: checking the ignoreExtension option") { withTempPath { tempDir => val df = spark.read.avro(episodesAvro) assert(df.count == 8) @@ -866,7 +866,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } - test("ignoreExtension must override hadoop's config") { + test("SPARK-24836: ignoreExtension must override hadoop's config") { withTempDir { dir => Files.copy( Paths.get(new URL(episodesAvro).toURI), From 3bd3475f1b4ca67b3f98c25a480c670f4fbb7a43 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Jul 2018 16:31:51 +0200 Subject: [PATCH 5/7] Moving the ignoreExtension to AvroOptions --- .../org/apache/spark/sql/avro/AvroFileFormat.scala | 11 ++++------- .../scala/org/apache/spark/sql/avro/AvroOptions.scala | 8 ++++++++ 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 234bd3623af8e..0c7340ba91326 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -63,7 +63,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Schema evolution is not supported yet. Here we only pick a single random sample file to // figure out the schema of the whole dataset. val sampleFile = - if (AvroFileFormat.ignoreExtension(conf, options)) { + if (AvroFileFormat.ignoreExtension(conf, parsedOptions)) { files.headOption.getOrElse { throw new FileNotFoundException("Files for schema inferring have been not found.") } @@ -169,7 +169,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. - if (AvroFileFormat.ignoreExtension(conf, options) || file.filePath.endsWith(".avro")) { + if (AvroFileFormat.ignoreExtension(conf, parsedOptions) || file.filePath.endsWith(".avro")) { val reader = { val in = new FsInput(new Path(new URI(file.filePath)), conf) try { @@ -273,15 +273,12 @@ private[avro] object AvroFileFormat { } } - def ignoreExtension(conf: Configuration, options: Map[String, String]): Boolean = { + def ignoreExtension(conf: Configuration, options: AvroOptions): Boolean = { val ignoreFilesWithoutExtensionByDefault = false val ignoreFilesWithoutExtension = conf.getBoolean( AvroFileFormat.IgnoreFilesWithoutExtensionProperty, ignoreFilesWithoutExtensionByDefault) - options - .get("ignoreExtension") - .map(_.toBoolean) - .getOrElse(!ignoreFilesWithoutExtension) + options.ignoreExtension.getOrElse(!ignoreFilesWithoutExtension) } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index 8721eae3481da..56a03b4c5b8c0 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -45,4 +45,12 @@ class AvroOptions(@transient val parameters: CaseInsensitiveMap[String]) * See Avro spec for details: https://avro.apache.org/docs/1.8.2/spec.html#schema_record . */ val recordNamespace: String = parameters.getOrElse("recordNamespace", "") + + /** + * The option controls ignoring of files without `.avro` extensions in read. + * If the option is enabled, all files (with and without `.avro` extension) are loaded. + * If the option is not set, the Hadoop's config `avro.mapred.ignore.inputs.without.extension` + * is taken into account. If the former one is not set too, file extensions are ignored. + */ + val ignoreExtension: Option[Boolean] = parameters.get("ignoreExtension").map(_.toBoolean) } From 0657508c7599a3a0ea70027bea96723e8088cc79 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Jul 2018 21:54:37 +0200 Subject: [PATCH 6/7] Moving ignoreExtension to AvroOptions --- .../spark/sql/avro/AvroFileFormat.scala | 19 ++++----------- .../apache/spark/sql/avro/AvroOptions.scala | 23 +++++++++++++++---- 2 files changed, 24 insertions(+), 18 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index 0c7340ba91326..ce31053027946 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -58,12 +58,12 @@ 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) + val parsedOptions = new AvroOptions(options, 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. val sampleFile = - if (AvroFileFormat.ignoreExtension(conf, parsedOptions)) { + if (parsedOptions.ignoreExtension) { files.headOption.getOrElse { throw new FileNotFoundException("Files for schema inferring have been not found.") } @@ -113,7 +113,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - val parsedOptions = new AvroOptions(options) + val parsedOptions = new AvroOptions(options, spark.sessionState.newHadoopConf()) val outputAvroSchema = SchemaConverters.toAvroType( dataSchema, nullable = false, parsedOptions.recordName, parsedOptions.recordNamespace) @@ -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) + val parsedOptions = new AvroOptions(options, hadoopConf) (file: PartitionedFile) => { val log = LoggerFactory.getLogger(classOf[AvroFileFormat]) @@ -169,7 +169,7 @@ private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { // Doing input file filtering is improper because we may generate empty tasks that process no // input files but stress the scheduler. We should probably add a more general input file // filtering mechanism for `FileFormat` data sources. See SPARK-16317. - if (AvroFileFormat.ignoreExtension(conf, parsedOptions) || file.filePath.endsWith(".avro")) { + if (parsedOptions.ignoreExtension || file.filePath.endsWith(".avro")) { val reader = { val in = new FsInput(new Path(new URI(file.filePath)), conf) try { @@ -272,13 +272,4 @@ private[avro] object AvroFileFormat { value.readFields(new DataInputStream(in)) } } - - def ignoreExtension(conf: Configuration, options: AvroOptions): Boolean = { - val ignoreFilesWithoutExtensionByDefault = false - val ignoreFilesWithoutExtension = conf.getBoolean( - AvroFileFormat.IgnoreFilesWithoutExtensionProperty, - ignoreFilesWithoutExtensionByDefault) - - options.ignoreExtension.getOrElse(!ignoreFilesWithoutExtension) - } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index 56a03b4c5b8c0..93a90b0b63f90 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -17,16 +17,21 @@ package org.apache.spark.sql.avro +import org.apache.hadoop.conf.Configuration + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap /** * Options for Avro Reader and Writer stored in case insensitive manner. */ -class AvroOptions(@transient val parameters: CaseInsensitiveMap[String]) - extends Logging with Serializable { +class AvroOptions( + @transient val parameters: CaseInsensitiveMap[String], + @transient val conf: Configuration) extends Logging with Serializable { - def this(parameters: Map[String, String]) = this(CaseInsensitiveMap(parameters)) + def this(parameters: Map[String, String], conf: Configuration) = { + this(CaseInsensitiveMap(parameters), conf) + } /** * Optional schema provided by an user in JSON format. @@ -52,5 +57,15 @@ class AvroOptions(@transient val parameters: CaseInsensitiveMap[String]) * If the option is not set, the Hadoop's config `avro.mapred.ignore.inputs.without.extension` * is taken into account. If the former one is not set too, file extensions are ignored. */ - val ignoreExtension: Option[Boolean] = parameters.get("ignoreExtension").map(_.toBoolean) + val ignoreExtension: Boolean = { + val ignoreFilesWithoutExtensionByDefault = false + val ignoreFilesWithoutExtension = conf.getBoolean( + AvroFileFormat.IgnoreFilesWithoutExtensionProperty, + ignoreFilesWithoutExtensionByDefault) + + parameters + .get("ignoreExtension") + .map(_.toBoolean) + .getOrElse(!ignoreFilesWithoutExtension) + } } From 3206a20fc9f3036e16eca20118e1559d722ff0b9 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Jul 2018 22:02:42 +0200 Subject: [PATCH 7/7] Adding option name to comment --- .../src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index 93a90b0b63f90..cd9a911a14bfa 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -52,7 +52,7 @@ class AvroOptions( val recordNamespace: String = parameters.getOrElse("recordNamespace", "") /** - * The option controls ignoring of files without `.avro` extensions in read. + * The `ignoreExtension` option controls ignoring of files without `.avro` extensions in read. * If the option is enabled, all files (with and without `.avro` extension) are loaded. * If the option is not set, the Hadoop's config `avro.mapred.ignore.inputs.without.extension` * is taken into account. If the former one is not set too, file extensions are ignored.