-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-24768][SQL] Have a built-in AVRO data source implementation #21742
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
Closed
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
<!-- | ||
~ 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. | ||
--> | ||
|
||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
<parent> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-parent_2.11</artifactId> | ||
<version>2.4.0-SNAPSHOT</version> | ||
<relativePath>../../pom.xml</relativePath> | ||
</parent> | ||
|
||
<artifactId>spark-sql-avro_2.11</artifactId> | ||
<properties> | ||
<sbt.project.name>avro</sbt.project.name> | ||
</properties> | ||
<packaging>jar</packaging> | ||
<name>Spark Avro</name> | ||
<url>http://spark.apache.org/</url> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
<scope>provided</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-core_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-catalyst_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-sql_${scala.binary.version}</artifactId> | ||
<version>${project.version}</version> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
<dependency> | ||
<groupId>org.apache.spark</groupId> | ||
<artifactId>spark-tags_${scala.binary.version}</artifactId> | ||
</dependency> | ||
</dependencies> | ||
<build> | ||
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory> | ||
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory> | ||
</build> | ||
</project> |
1 change: 1 addition & 0 deletions
1
...avro/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1 @@ | ||
org.apache.spark.sql.avro.AvroFileFormat |
289 changes: 289 additions & 0 deletions
289
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,289 @@ | ||
/* | ||
* 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.avro | ||
|
||
import java.io._ | ||
import java.net.URI | ||
import java.util.zip.Deflater | ||
|
||
import scala.util.control.NonFatal | ||
|
||
import com.esotericsoftware.kryo.{Kryo, KryoSerializable} | ||
import com.esotericsoftware.kryo.io.{Input, Output} | ||
import org.apache.avro.{Schema, SchemaBuilder} | ||
import org.apache.avro.file.{DataFileConstants, DataFileReader} | ||
import org.apache.avro.generic.{GenericDatumReader, GenericRecord} | ||
import org.apache.avro.mapred.{AvroOutputFormat, FsInput} | ||
import org.apache.avro.mapreduce.AvroJob | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.{FileStatus, Path} | ||
import org.apache.hadoop.mapreduce.Job | ||
import org.slf4j.LoggerFactory | ||
|
||
import org.apache.spark.TaskContext | ||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.encoders.RowEncoder | ||
import org.apache.spark.sql.catalyst.expressions.GenericRow | ||
import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} | ||
import org.apache.spark.sql.sources.{DataSourceRegister, Filter} | ||
import org.apache.spark.sql.types.StructType | ||
|
||
private[avro] class AvroFileFormat extends FileFormat with DataSourceRegister { | ||
private val log = LoggerFactory.getLogger(getClass) | ||
|
||
override def equals(other: Any): Boolean = other match { | ||
case _: AvroFileFormat => true | ||
case _ => false | ||
} | ||
|
||
// Dummy hashCode() to appease ScalaStyle. | ||
override def hashCode(): Int = super.hashCode() | ||
|
||
override def inferSchema( | ||
spark: SparkSession, | ||
options: Map[String, String], | ||
files: Seq[FileStatus]): Option[StructType] = { | ||
val conf = spark.sparkContext.hadoopConfiguration | ||
|
||
// 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 (conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, true)) { | ||
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?" | ||
) | ||
} | ||
} else { | ||
files.headOption.getOrElse { | ||
throw new FileNotFoundException("No Avro files found.") | ||
} | ||
} | ||
|
||
// User can specify an optional avro json schema. | ||
val avroSchema = options.get(AvroFileFormat.AvroSchema) | ||
.map(new Schema.Parser().parse) | ||
.getOrElse { | ||
val in = new FsInput(sampleFile.getPath, conf) | ||
try { | ||
val reader = DataFileReader.openReader(in, new GenericDatumReader[GenericRecord]()) | ||
try { | ||
reader.getSchema | ||
} finally { | ||
reader.close() | ||
} | ||
} finally { | ||
in.close() | ||
} | ||
} | ||
|
||
SchemaConverters.toSqlType(avroSchema).dataType match { | ||
case t: StructType => Some(t) | ||
case _ => throw new RuntimeException( | ||
s"""Avro schema cannot be converted to a Spark SQL StructType: | ||
| | ||
|${avroSchema.toString(true)} | ||
|""".stripMargin) | ||
} | ||
} | ||
|
||
override def shortName(): String = "avro" | ||
|
||
override def isSplitable( | ||
sparkSession: SparkSession, | ||
options: Map[String, String], | ||
path: Path): Boolean = true | ||
|
||
override def prepareWrite( | ||
spark: SparkSession, | ||
job: Job, | ||
options: Map[String, String], | ||
dataSchema: StructType): OutputWriterFactory = { | ||
val recordName = options.getOrElse("recordName", "topLevelRecord") | ||
val recordNamespace = options.getOrElse("recordNamespace", "") | ||
val build = SchemaBuilder.record(recordName).namespace(recordNamespace) | ||
val outputAvroSchema = SchemaConverters.convertStructToAvro(dataSchema, build, 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 { | ||
case "uncompressed" => | ||
log.info("writing uncompressed Avro records") | ||
job.getConfiguration.setBoolean(COMPRESS_KEY, false) | ||
|
||
case "snappy" => | ||
log.info("compressing Avro output using Snappy") | ||
job.getConfiguration.setBoolean(COMPRESS_KEY, true) | ||
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 | ||
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) | ||
job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel) | ||
|
||
case unknown: String => | ||
log.error(s"unsupported compression codec $unknown") | ||
} | ||
|
||
new AvroOutputWriterFactory(dataSchema, recordName, recordNamespace) | ||
} | ||
|
||
override def buildReader( | ||
spark: SparkSession, | ||
dataSchema: StructType, | ||
partitionSchema: StructType, | ||
requiredSchema: StructType, | ||
filters: Seq[Filter], | ||
options: Map[String, String], | ||
hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { | ||
|
||
val broadcastedConf = | ||
spark.sparkContext.broadcast(new AvroFileFormat.SerializableConfiguration(hadoopConf)) | ||
|
||
(file: PartitionedFile) => { | ||
val log = LoggerFactory.getLogger(classOf[AvroFileFormat]) | ||
val conf = broadcastedConf.value.value | ||
val userProvidedSchema = options.get(AvroFileFormat.AvroSchema).map(new Schema.Parser().parse) | ||
|
||
// TODO Removes this check once `FileFormat` gets a general file filtering interface method. | ||
// 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 ( | ||
conf.getBoolean(AvroFileFormat.IgnoreFilesWithoutExtensionProperty, true) && | ||
!file.filePath.endsWith(".avro") | ||
) { | ||
Iterator.empty | ||
} else { | ||
val reader = { | ||
val in = new FsInput(new Path(new URI(file.filePath)), conf) | ||
try { | ||
val datumReader = userProvidedSchema match { | ||
case Some(userSchema) => new GenericDatumReader[GenericRecord](userSchema) | ||
case _ => new GenericDatumReader[GenericRecord]() | ||
} | ||
DataFileReader.openReader(in, datumReader) | ||
} catch { | ||
case NonFatal(e) => | ||
log.error("Exception while opening DataFileReader", e) | ||
in.close() | ||
throw e | ||
} | ||
} | ||
|
||
// Ensure that the reader is closed even if the task fails or doesn't consume the entire | ||
// iterator of records. | ||
Option(TaskContext.get()).foreach { taskContext => | ||
taskContext.addTaskCompletionListener { _ => | ||
reader.close() | ||
} | ||
} | ||
|
||
reader.sync(file.start) | ||
val stop = file.start + file.length | ||
|
||
val rowConverter = SchemaConverters.createConverterToSQL( | ||
userProvidedSchema.getOrElse(reader.getSchema), requiredSchema) | ||
|
||
new Iterator[InternalRow] { | ||
// Used to convert `Row`s containing data columns into `InternalRow`s. | ||
private val encoderForDataColumns = RowEncoder(requiredSchema) | ||
|
||
private[this] var completed = false | ||
|
||
override def hasNext: Boolean = { | ||
if (completed) { | ||
false | ||
} else { | ||
val r = reader.hasNext && !reader.pastSync(stop) | ||
if (!r) { | ||
reader.close() | ||
completed = true | ||
} | ||
r | ||
} | ||
} | ||
|
||
override def next(): InternalRow = { | ||
if (reader.pastSync(stop)) { | ||
throw new NoSuchElementException("next on empty iterator") | ||
} | ||
val record = reader.next() | ||
val safeDataRow = rowConverter(record).asInstanceOf[GenericRow] | ||
|
||
// The safeDataRow is reused, we must do a copy | ||
encoderForDataColumns.toRow(safeDataRow) | ||
} | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
private[avro] object AvroFileFormat { | ||
val IgnoreFilesWithoutExtensionProperty = "avro.mapred.ignore.inputs.without.extension" | ||
|
||
val AvroSchema = "avroSchema" | ||
|
||
class SerializableConfiguration(@transient var value: Configuration) | ||
extends Serializable with KryoSerializable { | ||
@transient private[avro] lazy val log = LoggerFactory.getLogger(getClass) | ||
|
||
private def writeObject(out: ObjectOutputStream): Unit = tryOrIOException { | ||
out.defaultWriteObject() | ||
value.write(out) | ||
} | ||
|
||
private def readObject(in: ObjectInputStream): Unit = tryOrIOException { | ||
value = new Configuration(false) | ||
value.readFields(in) | ||
} | ||
|
||
private def tryOrIOException[T](block: => T): T = { | ||
try { | ||
block | ||
} catch { | ||
case e: IOException => | ||
log.error("Exception encountered", e) | ||
throw e | ||
case NonFatal(e) => | ||
log.error("Exception encountered", e) | ||
throw new IOException(e) | ||
} | ||
} | ||
|
||
def write(kryo: Kryo, out: Output): Unit = { | ||
val dos = new DataOutputStream(out) | ||
value.write(dos) | ||
dos.flush() | ||
} | ||
|
||
def read(kryo: Kryo, in: Input): Unit = { | ||
value = new Configuration(false) | ||
value.readFields(new DataInputStream(in)) | ||
} | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Why does this need a separate module unlike other datasources?
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.
This is much cleaner, like what we did for kafka, which is also a built-in data source. Ideally, we should separate parquet, orc and other built-in data sources from sql module. We can do the refactoring in the future, if needed