Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
add from_avro and to_avro
  • Loading branch information
gengliangwang committed Jul 21, 2018
commit b02d7edbf39f9ad4c4a2fdae6eeec1104097057e
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* 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.generic.GenericDatumReader
import org.apache.avro.io.{BinaryDecoder, DecoderFactory}

import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters, SerializableSchema}
import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType}

case class AvroDataToCatalyst(child: Expression, avroType: SerializableSchema)
extends UnaryExpression with CodegenFallback with ExpectsInputTypes {

override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType)

override lazy val dataType: DataType =
SchemaConverters.toSqlType(avroType.value).dataType

override def nullable: Boolean = true

@transient private lazy val reader = new GenericDatumReader[Any](avroType.value)

@transient private lazy val deserializer = new AvroDeserializer(avroType.value, 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 = simpleString
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.CodegenFallback
import org.apache.spark.sql.types.{BinaryType, DataType}

case class CatalystDataToAvro(child: Expression) extends UnaryExpression with CodegenFallback {

override lazy val 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 = simpleString
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,10 @@

package org.apache.spark.sql

import org.apache.avro.Schema

import org.apache.spark.annotation.Experimental

package object avro {
/**
* Adds a method, `avro`, to DataFrameWriter that allows you to write avro files using
Expand All @@ -36,4 +40,27 @@ package object avro {
@scala.annotation.varargs
def avro(sources: String*): DataFrame = reader.format("avro").load(sources: _*)
}

/**
* Converts a binary column of avro format into its corresponding catalyst value. The specified
* schema must match the read data, otherwise the behavior is undefined: it may fail or return
* arbitrary result.
*
* @param data the binary column.
* @param avroType the avro type.
*/
@Experimental
def from_avro(data: Column, avroType: Schema): Column = {
new Column(AvroDataToCatalyst(data.expr, new SerializableSchema(avroType)))
}

/**
* Converts a column into binary of avro format.
*
* @param data the data column.
*/
@Experimental
def to_avro(data: Column): Column = {
new Column(CatalystDataToAvro(data.expr))
}
}