forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 4
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[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)
- Loading branch information
1 parent
66f45bf
commit d6a633f
Showing
8 changed files
with
447 additions
and
32 deletions.
There are no files selected for viewing
This file contains 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
68 changes: 68 additions & 0 deletions
68
external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala
This file contains 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,68 @@ | ||
/* | ||
* 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 | ||
|
||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericDatumReader | ||
import org.apache.avro.io.{BinaryDecoder, DecoderFactory} | ||
|
||
import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters} | ||
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} | ||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} | ||
import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} | ||
|
||
case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) | ||
extends UnaryExpression with ExpectsInputTypes { | ||
|
||
override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) | ||
|
||
override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType | ||
|
||
override def nullable: Boolean = true | ||
|
||
@transient private lazy val avroSchema = new Schema.Parser().parse(jsonFormatSchema) | ||
|
||
@transient private lazy val reader = new GenericDatumReader[Any](avroSchema) | ||
|
||
@transient private lazy val deserializer = new AvroDeserializer(avroSchema, dataType) | ||
|
||
@transient private var decoder: BinaryDecoder = _ | ||
|
||
@transient private var result: Any = _ | ||
|
||
override def nullSafeEval(input: Any): Any = { | ||
val binary = input.asInstanceOf[Array[Byte]] | ||
decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) | ||
result = reader.read(result, decoder) | ||
deserializer.deserialize(result) | ||
} | ||
|
||
override def simpleString: String = { | ||
s"from_avro(${child.sql}, ${dataType.simpleString})" | ||
} | ||
|
||
override def sql: String = { | ||
s"from_avro(${child.sql}, ${dataType.catalogString})" | ||
} | ||
|
||
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { | ||
val expr = ctx.addReferenceObj("this", this) | ||
defineCodeGen(ctx, ev, input => | ||
s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") | ||
} | ||
} |
69 changes: 69 additions & 0 deletions
69
external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala
This file contains 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,69 @@ | ||
/* | ||
* 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 | ||
|
||
import java.io.ByteArrayOutputStream | ||
|
||
import org.apache.avro.generic.GenericDatumWriter | ||
import org.apache.avro.io.{BinaryEncoder, EncoderFactory} | ||
|
||
import org.apache.spark.sql.avro.{AvroSerializer, SchemaConverters} | ||
import org.apache.spark.sql.catalyst.expressions.{Expression, UnaryExpression} | ||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} | ||
import org.apache.spark.sql.types.{BinaryType, DataType} | ||
|
||
case class CatalystDataToAvro(child: Expression) extends UnaryExpression { | ||
|
||
override def dataType: DataType = BinaryType | ||
|
||
@transient private lazy val avroType = | ||
SchemaConverters.toAvroType(child.dataType, child.nullable) | ||
|
||
@transient private lazy val serializer = | ||
new AvroSerializer(child.dataType, avroType, child.nullable) | ||
|
||
@transient private lazy val writer = | ||
new GenericDatumWriter[Any](avroType) | ||
|
||
@transient private var encoder: BinaryEncoder = _ | ||
|
||
@transient private lazy val out = new ByteArrayOutputStream | ||
|
||
override def nullSafeEval(input: Any): Any = { | ||
out.reset() | ||
encoder = EncoderFactory.get().directBinaryEncoder(out, encoder) | ||
val avroData = serializer.serialize(input) | ||
writer.write(avroData, encoder) | ||
encoder.flush() | ||
out.toByteArray | ||
} | ||
|
||
override def simpleString: String = { | ||
s"to_avro(${child.sql}, ${child.dataType.simpleString})" | ||
} | ||
|
||
override def sql: String = { | ||
s"to_avro(${child.sql}, ${child.dataType.catalogString})" | ||
} | ||
|
||
override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { | ||
val expr = ctx.addReferenceObj("this", this) | ||
defineCodeGen(ctx, ev, input => | ||
s"(byte[]) $expr.nullSafeEval($input)") | ||
} | ||
} |
This file contains 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 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
Oops, something went wrong.