Skip to content

Commit dae6c6f

Browse files
committed
[SC-5606] Inline spark-avro sources into databricks/spark
This patch ports `spark-avro` as of databricks/spark-avro@b01a034 and updates it to run with Spark 2.10 by including the fixes from databricks/spark-avro#206. Via a transitive dependency on `spark-core` and `spark-sql`, this winds up not needing to add new dependencies on Avro to the poms. I've updated the license headers to use the header that we use for Spark-edge features. Author: Josh Rosen <[email protected]> Closes apache#170 from JoshRosen/add-spark-avro.
1 parent d058ab5 commit dae6c6f

33 files changed

+2236
-7
lines changed

dev/run-tests.py

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -110,8 +110,8 @@ def determine_modules_to_test(changed_modules):
110110
['graphx', 'examples']
111111
>>> x = [x.name for x in determine_modules_to_test([modules.sql])]
112112
>>> x # doctest: +NORMALIZE_WHITESPACE
113-
['sql', 'hive', 'mllib', 'sql-kafka-0-10', 'sql-kafka-0-8', 'examples', 'hive-thriftserver',
114-
'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
113+
['sql', 'avro', 'hive', 'mllib', 'sql-kafka-0-10', 'sql-kafka-0-8', 'examples',
114+
'hive-thriftserver', 'pyspark-sql', 'sparkr', 'pyspark-mllib', 'pyspark-ml']
115115
"""
116116
modules_to_test = set()
117117
for module in changed_modules:

dev/sparktestsupport/modules.py

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -157,6 +157,16 @@ def __hash__(self):
157157
]
158158
)
159159

160+
avro = Module(
161+
name="avro",
162+
dependencies=[sql],
163+
source_file_regexes=[
164+
"external/avro",
165+
],
166+
sbt_test_goals=[
167+
"avro/test",
168+
]
169+
)
160170

161171
sql_kafka = Module(
162172
name="sql-kafka-0-10",

external/avro/pom.xml

Lines changed: 74 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,74 @@
1+
<?xml version="1.0" encoding="UTF-8"?>
2+
<!--
3+
~ Licensed to the Apache Software Foundation (ASF) under one or more
4+
~ contributor license agreements. See the NOTICE file distributed with
5+
~ this work for additional information regarding copyright ownership.
6+
~ The ASF licenses this file to You under the Apache License, Version 2.0
7+
~ (the "License"); you may not use this file except in compliance with
8+
~ the License. You may obtain a copy of the License at
9+
~
10+
~ http://www.apache.org/licenses/LICENSE-2.0
11+
~
12+
~ Unless required by applicable law or agreed to in writing, software
13+
~ distributed under the License is distributed on an "AS IS" BASIS,
14+
~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
~ See the License for the specific language governing permissions and
16+
~ limitations under the License.
17+
-->
18+
19+
<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">
20+
<modelVersion>4.0.0</modelVersion>
21+
<parent>
22+
<groupId>org.apache.spark</groupId>
23+
<artifactId>spark-parent_2.11</artifactId>
24+
<version>2.1.0</version>
25+
<relativePath>../../pom.xml</relativePath>
26+
</parent>
27+
28+
<groupId>com.databricks</groupId>
29+
<artifactId>spark-avro_2.11</artifactId>
30+
<properties>
31+
<sbt.project.name>avro</sbt.project.name>
32+
</properties>
33+
<packaging>jar</packaging>
34+
<name>Spark Avro</name>
35+
<url>http://spark.apache.org/</url>
36+
37+
<dependencies>
38+
<dependency>
39+
<groupId>org.apache.spark</groupId>
40+
<artifactId>spark-sql_${scala.binary.version}</artifactId>
41+
<version>${project.version}</version>
42+
<scope>provided</scope>
43+
</dependency>
44+
<dependency>
45+
<groupId>org.apache.spark</groupId>
46+
<artifactId>spark-core_${scala.binary.version}</artifactId>
47+
<version>${project.version}</version>
48+
<type>test-jar</type>
49+
<scope>test</scope>
50+
</dependency>
51+
<dependency>
52+
<groupId>org.apache.spark</groupId>
53+
<artifactId>spark-catalyst_${scala.binary.version}</artifactId>
54+
<version>${project.version}</version>
55+
<type>test-jar</type>
56+
<scope>test</scope>
57+
</dependency>
58+
<dependency>
59+
<groupId>org.apache.spark</groupId>
60+
<artifactId>spark-sql_${scala.binary.version}</artifactId>
61+
<version>${project.version}</version>
62+
<type>test-jar</type>
63+
<scope>test</scope>
64+
</dependency>
65+
<dependency>
66+
<groupId>org.apache.spark</groupId>
67+
<artifactId>spark-tags_${scala.binary.version}</artifactId>
68+
</dependency>
69+
</dependencies>
70+
<build>
71+
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
72+
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
73+
</build>
74+
</project>
Lines changed: 140 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,140 @@
1+
/*
2+
* Copyright (C) 2016 Databricks, Inc.
3+
*
4+
* Portions of this software incorporate or are derived from software contained within Apache Spark,
5+
* and this modified software differs from the Apache Spark software provided under the Apache
6+
* License, Version 2.0, a copy of which you may obtain at
7+
* http://www.apache.org/licenses/LICENSE-2.0
8+
*/
9+
10+
package com.databricks.spark.avro
11+
12+
import java.io.{IOException, OutputStream}
13+
import java.nio.ByteBuffer
14+
import java.sql.Timestamp
15+
import java.util.HashMap
16+
17+
import scala.collection.immutable.Map
18+
19+
import org.apache.avro.{Schema, SchemaBuilder}
20+
import org.apache.avro.generic.GenericData.Record
21+
import org.apache.avro.generic.GenericRecord
22+
import org.apache.avro.mapred.AvroKey
23+
import org.apache.avro.mapreduce.AvroKeyOutputFormat
24+
import org.apache.hadoop.fs.Path
25+
import org.apache.hadoop.io.NullWritable
26+
import org.apache.hadoop.mapreduce.{RecordWriter, TaskAttemptContext}
27+
28+
import org.apache.spark.sql.Row
29+
import org.apache.spark.sql.execution.datasources.OutputWriter
30+
import org.apache.spark.sql.types._
31+
32+
// NOTE: This class is instantiated and used on executor side only, no need to be serializable.
33+
private[avro] class AvroOutputWriter(
34+
path: String,
35+
context: TaskAttemptContext,
36+
schema: StructType,
37+
recordName: String,
38+
recordNamespace: String) extends OutputWriter {
39+
40+
private lazy val converter = createConverterToAvro(schema, recordName, recordNamespace)
41+
42+
/**
43+
* Overrides the couple of methods responsible for generating the output streams / files so
44+
* that the data can be correctly partitioned
45+
*/
46+
private val recordWriter: RecordWriter[AvroKey[GenericRecord], NullWritable] =
47+
new AvroKeyOutputFormat[GenericRecord]() {
48+
49+
override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
50+
new Path(path)
51+
}
52+
53+
@throws(classOf[IOException])
54+
override def getAvroFileOutputStream(c: TaskAttemptContext): OutputStream = {
55+
val path = getDefaultWorkFile(context, ".avro")
56+
path.getFileSystem(context.getConfiguration).create(path)
57+
}
58+
59+
}.getRecordWriter(context)
60+
61+
override def write(row: Row): Unit = {
62+
val key = new AvroKey(converter(row).asInstanceOf[GenericRecord])
63+
recordWriter.write(key, NullWritable.get())
64+
}
65+
66+
override def close(): Unit = recordWriter.close(context)
67+
68+
/**
69+
* This function constructs converter function for a given sparkSQL datatype. This is used in
70+
* writing Avro records out to disk
71+
*/
72+
private def createConverterToAvro(
73+
dataType: DataType,
74+
structName: String,
75+
recordNamespace: String): (Any) => Any = {
76+
dataType match {
77+
case BinaryType => (item: Any) => item match {
78+
case null => null
79+
case bytes: Array[Byte] => ByteBuffer.wrap(bytes)
80+
}
81+
case ByteType | ShortType | IntegerType | LongType |
82+
FloatType | DoubleType | StringType | BooleanType => identity
83+
case _: DecimalType => (item: Any) => if (item == null) null else item.toString
84+
case TimestampType => (item: Any) =>
85+
if (item == null) null else item.asInstanceOf[Timestamp].getTime
86+
case ArrayType(elementType, _) =>
87+
val elementConverter = createConverterToAvro(elementType, structName, recordNamespace)
88+
(item: Any) => {
89+
if (item == null) {
90+
null
91+
} else {
92+
val sourceArray = item.asInstanceOf[Seq[Any]]
93+
val sourceArraySize = sourceArray.size
94+
val targetArray = new Array[Any](sourceArraySize)
95+
var idx = 0
96+
while (idx < sourceArraySize) {
97+
targetArray(idx) = elementConverter(sourceArray(idx))
98+
idx += 1
99+
}
100+
targetArray
101+
}
102+
}
103+
case MapType(StringType, valueType, _) =>
104+
val valueConverter = createConverterToAvro(valueType, structName, recordNamespace)
105+
(item: Any) => {
106+
if (item == null) {
107+
null
108+
} else {
109+
val javaMap = new HashMap[String, Any]()
110+
item.asInstanceOf[Map[String, Any]].foreach { case (key, value) =>
111+
javaMap.put(key, valueConverter(value))
112+
}
113+
javaMap
114+
}
115+
}
116+
case structType: StructType =>
117+
val builder = SchemaBuilder.record(structName).namespace(recordNamespace)
118+
val schema: Schema = SchemaConverters.convertStructToAvro(
119+
structType, builder, recordNamespace)
120+
val fieldConverters = structType.fields.map(field =>
121+
createConverterToAvro(field.dataType, field.name, recordNamespace))
122+
(item: Any) => {
123+
if (item == null) {
124+
null
125+
} else {
126+
val record = new Record(schema)
127+
val convertersIterator = fieldConverters.iterator
128+
val fieldNamesIterator = dataType.asInstanceOf[StructType].fieldNames.iterator
129+
val rowIterator = item.asInstanceOf[Row].toSeq.iterator
130+
131+
while (convertersIterator.hasNext) {
132+
val converter = convertersIterator.next()
133+
record.put(fieldNamesIterator.next(), converter(rowIterator.next()))
134+
}
135+
record
136+
}
137+
}
138+
}
139+
}
140+
}
Lines changed: 30 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,30 @@
1+
/*
2+
* Copyright (C) 2016 Databricks, Inc.
3+
*
4+
* Portions of this software incorporate or are derived from software contained within Apache Spark,
5+
* and this modified software differs from the Apache Spark software provided under the Apache
6+
* License, Version 2.0, a copy of which you may obtain at
7+
* http://www.apache.org/licenses/LICENSE-2.0
8+
*/
9+
10+
package com.databricks.spark.avro
11+
12+
import org.apache.hadoop.mapreduce.TaskAttemptContext
13+
14+
import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory}
15+
import org.apache.spark.sql.types.StructType
16+
17+
private[avro] class AvroOutputWriterFactory(
18+
schema: StructType,
19+
recordName: String,
20+
recordNamespace: String) extends OutputWriterFactory {
21+
22+
override def getFileExtension(context: TaskAttemptContext): String = ".avro"
23+
24+
def newInstance(
25+
path: String,
26+
dataSchema: StructType,
27+
context: TaskAttemptContext): OutputWriter = {
28+
new AvroOutputWriter(path, context, schema, recordName, recordNamespace)
29+
}
30+
}

0 commit comments

Comments
 (0)