Skip to content

Commit d6a633f

Browse files
gengliangwangYuxiang Chen
authored andcommitted
[SPARK-24811][SQL] Avro: add new function from_avro and to_avro
1. Add a new function from_avro for parsing a binary column of avro format and converting it into its corresponding catalyst value. 2. Add a new function to_avro for converting a column into binary of avro format with the specified schema. I created apache#21774 for this, but it failed the build https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.6/7902/ Additional changes In this PR: 1. Add `scalacheck` dependency in pom.xml to resolve the failure. 2. Update the `log4j.properties` to make it consistent with other modules. Unit test Compile with different commands: ``` ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.6 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-2.7 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ./build/mvn --force -DzincPort=3643 -DskipTests -Phadoop-3.1 -Phive-thriftserver -Pkinesis-asl -Pspark-ganglia-lgpl -Pmesos -Pyarn compile test-compile ``` Author: Gengliang Wang <[email protected]> Closes apache#21838 from gengliangwang/from_and_to_avro. (cherry picked from commit 8817c68)
1 parent 66f45bf commit d6a633f

File tree

8 files changed

+447
-32
lines changed

8 files changed

+447
-32
lines changed

external/avro/pom.xml

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -61,6 +61,11 @@
6161
<type>test-jar</type>
6262
<scope>test</scope>
6363
</dependency>
64+
<dependency>
65+
<groupId>org.scalacheck</groupId>
66+
<artifactId>scalacheck_${scala.binary.version}</artifactId>
67+
<scope>test</scope>
68+
</dependency>
6469
<dependency>
6570
<groupId>org.apache.spark</groupId>
6671
<artifactId>spark-tags_${scala.binary.version}</artifactId>
Lines changed: 68 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,68 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
import org.apache.avro.Schema
21+
import org.apache.avro.generic.GenericDatumReader
22+
import org.apache.avro.io.{BinaryDecoder, DecoderFactory}
23+
24+
import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters}
25+
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression}
26+
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
27+
import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType}
28+
29+
case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String)
30+
extends UnaryExpression with ExpectsInputTypes {
31+
32+
override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)
33+
34+
override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType
35+
36+
override def nullable: Boolean = true
37+
38+
@transient private lazy val avroSchema = new Schema.Parser().parse(jsonFormatSchema)
39+
40+
@transient private lazy val reader = new GenericDatumReader[Any](avroSchema)
41+
42+
@transient private lazy val deserializer = new AvroDeserializer(avroSchema, dataType)
43+
44+
@transient private var decoder: BinaryDecoder = _
45+
46+
@transient private var result: Any = _
47+
48+
override def nullSafeEval(input: Any): Any = {
49+
val binary = input.asInstanceOf[Array[Byte]]
50+
decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder)
51+
result = reader.read(result, decoder)
52+
deserializer.deserialize(result)
53+
}
54+
55+
override def simpleString: String = {
56+
s"from_avro(${child.sql}, ${dataType.simpleString})"
57+
}
58+
59+
override def sql: String = {
60+
s"from_avro(${child.sql}, ${dataType.catalogString})"
61+
}
62+
63+
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
64+
val expr = ctx.addReferenceObj("this", this)
65+
defineCodeGen(ctx, ev, input =>
66+
s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)")
67+
}
68+
}
Lines changed: 69 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,69 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
import java.io.ByteArrayOutputStream
21+
22+
import org.apache.avro.generic.GenericDatumWriter
23+
import org.apache.avro.io.{BinaryEncoder, EncoderFactory}
24+
25+
import org.apache.spark.sql.avro.{AvroSerializer, SchemaConverters}
26+
import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression}
27+
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode}
28+
import org.apache.spark.sql.types.{BinaryType, DataType}
29+
30+
case class CatalystDataToAvro(child: Expression) extends UnaryExpression {
31+
32+
override def dataType: DataType = BinaryType
33+
34+
@transient private lazy val avroType =
35+
SchemaConverters.toAvroType(child.dataType, child.nullable)
36+
37+
@transient private lazy val serializer =
38+
new AvroSerializer(child.dataType, avroType, child.nullable)
39+
40+
@transient private lazy val writer =
41+
new GenericDatumWriter[Any](avroType)
42+
43+
@transient private var encoder: BinaryEncoder = _
44+
45+
@transient private lazy val out = new ByteArrayOutputStream
46+
47+
override def nullSafeEval(input: Any): Any = {
48+
out.reset()
49+
encoder = EncoderFactory.get().directBinaryEncoder(out, encoder)
50+
val avroData = serializer.serialize(input)
51+
writer.write(avroData, encoder)
52+
encoder.flush()
53+
out.toByteArray
54+
}
55+
56+
override def simpleString: String = {
57+
s"to_avro(${child.sql}, ${child.dataType.simpleString})"
58+
}
59+
60+
override def sql: String = {
61+
s"to_avro(${child.sql}, ${child.dataType.catalogString})"
62+
}
63+
64+
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
65+
val expr = ctx.addReferenceObj("this", this)
66+
defineCodeGen(ctx, ev, input =>
67+
s"(byte[]) $expr.nullSafeEval($input)")
68+
}
69+
}

external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala

Lines changed: 31 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,10 @@
1717

1818
package org.apache.spark.sql
1919

20+
import org.apache.avro.Schema
21+
22+
import org.apache.spark.annotation.Experimental
23+
2024
package object avro {
2125
/**
2226
* Adds a method, `avro`, to DataFrameWriter that allows you to write avro files using
@@ -36,4 +40,31 @@ package object avro {
3640
@scala.annotation.varargs
3741
def avro(sources: String*): DataFrame = reader.format("avro").load(sources: _*)
3842
}
43+
44+
/**
45+
* Converts a binary column of avro format into its corresponding catalyst value. The specified
46+
* schema must match the read data, otherwise the behavior is undefined: it may fail or return
47+
* arbitrary result.
48+
*
49+
* @param data the binary column.
50+
* @param jsonFormatSchema the avro schema in JSON string format.
51+
*
52+
* @since 2.4.0
53+
*/
54+
@Experimental
55+
def from_avro(data: Column, jsonFormatSchema: String): Column = {
56+
new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema))
57+
}
58+
59+
/**
60+
* Converts a column into binary of avro format.
61+
*
62+
* @param data the data column.
63+
*
64+
* @since 2.4.0
65+
*/
66+
@Experimental
67+
def to_avro(data: Column): Column = {
68+
new Column(CatalystDataToAvro(data.expr))
69+
}
3970
}

external/avro/src/test/resources/log4j.properties

Lines changed: 10 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -15,35 +15,13 @@
1515
# limitations under the License.
1616
#
1717

18-
# Set everything to be logged to the file core/target/unit-tests.log
19-
log4j.rootLogger=DEBUG, CA, FA
20-
21-
#Console Appender
22-
log4j.appender.CA=org.apache.log4j.ConsoleAppender
23-
log4j.appender.CA.layout=org.apache.log4j.PatternLayout
24-
log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n
25-
log4j.appender.CA.Threshold = WARN
26-
27-
28-
#File Appender
29-
log4j.appender.FA=org.apache.log4j.FileAppender
30-
log4j.appender.FA.append=false
31-
log4j.appender.FA.file=target/unit-tests.log
32-
log4j.appender.FA.layout=org.apache.log4j.PatternLayout
33-
log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n
34-
35-
# Set the logger level of File Appender to WARN
36-
log4j.appender.FA.Threshold = INFO
37-
38-
# Some packages are noisy for no good reason.
39-
log4j.additivity.parquet.hadoop.ParquetRecordReader=false
40-
log4j.logger.parquet.hadoop.ParquetRecordReader=OFF
41-
42-
log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false
43-
log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF
44-
45-
log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false
46-
log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF
47-
48-
log4j.additivity.hive.ql.metadata.Hive=false
49-
log4j.logger.hive.ql.metadata.Hive=OFF
18+
# Set everything to be logged to the file target/unit-tests.log
19+
log4j.rootCategory=INFO, file
20+
log4j.appender.file=org.apache.log4j.FileAppender
21+
log4j.appender.file.append=true
22+
log4j.appender.file.file=target/unit-tests.log
23+
log4j.appender.file.layout=org.apache.log4j.PatternLayout
24+
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
25+
26+
# Ignore messages below warning level from Jetty, because it's a bit verbose
27+
log4j.logger.org.spark-project.jetty=WARN

0 commit comments

Comments
 (0)