You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/02/22 21:26:52 UTC

[GitHub] kunkun-tang commented on a change in pull request #22878: [SPARK-25789][SQL] Support for Dataset of Avro

kunkun-tang commented on a change in pull request #22878: [SPARK-25789][SQL] Support for Dataset of Avro
URL: https://github.com/apache/spark/pull/22878#discussion_r259515921
 
 

 ##########
 File path: external/avro/src/main/scala/org/apache/spark/sql/avro/AvroEncoder.scala
 ##########
 @@ -0,0 +1,549 @@
+/*
+ * 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.util.{Map => JMap}
+
+import scala.collection.JavaConverters._
+import scala.language.existentials
+import scala.reflect.ClassTag
+
+import org.apache.avro.Schema
+import org.apache.avro.Schema.Parser
+import org.apache.avro.Schema.Type._
+import org.apache.avro.generic.{GenericData, IndexedRecord}
+import org.apache.avro.reflect.ReflectData
+import org.apache.avro.specific.SpecificRecord
+
+import org.apache.spark.sql.Encoder
+import org.apache.spark.sql.avro.SchemaConverters._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedExtractValue}
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.expressions.objects.{LambdaVariable => _, _}
+import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData}
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * A Spark-SQL Encoder for Avro objects
+ */
+object AvroEncoder {
+  /**
+   * Provides an Encoder for Avro objects of the given class
+   *
+   * @param avroClass the class of the Avro object for which to generate the Encoder
+   * @tparam T the type of the Avro class, must implement SpecificRecord
+   * @return an Encoder for the given Avro class
+   */
+  def of[T <: SpecificRecord](avroClass: Class[T]): Encoder[T] = {
+    AvroExpressionEncoder.of(avroClass)
+  }
+
+  /**
+   * Provides an Encoder for Avro objects implementing the given schema
+   *
+   * @param avroSchema the Schema of the Avro object for which to generate the Encoder
+   * @tparam T the type of the Avro class that implements the Schema, must implement IndexedRecord
+   * @return an Encoder for the given Avro Schema
+   */
+  def of[T <: IndexedRecord](avroSchema: Schema): Encoder[T] = {
+    AvroExpressionEncoder.of(avroSchema)
+  }
+
+  /**
+   * Provides an Encoder for Avro objects implementing the given schema
+   *
+   * @param jsonFormatSchema the json string represented Schema of the Avro object
+   *                         for which to generate the Encoder
+   * @tparam T the type of the Avro class that implements the Schema, must implement IndexedRecord
+   * @return an Encoder for the given Avro Schema
+   */
+  def of[T <: IndexedRecord](jsonFormatSchema: String): Encoder[T] = {
+    val avroSchema = new Schema.Parser().parse(jsonFormatSchema)
+    AvroExpressionEncoder.of(avroSchema)
+  }
+}
+
+class SerializableSchema(@transient var value: Schema) extends Externalizable {
+  def this() = this(null)
+  override def readExternal(in: ObjectInput): Unit = {
+    value = new Parser().parse(in.readObject().asInstanceOf[String])
+  }
+  override def writeExternal(out: ObjectOutput): Unit = out.writeObject(value.toString)
+  def resolveUnion(datum: Any): Int = GenericData.get.resolveUnion(value, datum)
+}
+
+object AvroExpressionEncoder {
+
+  def of[T <: SpecificRecord](avroClass: Class[T]): ExpressionEncoder[T] = {
+    val schema = avroClass.getMethod("getClassSchema").invoke(null).asInstanceOf[Schema]
+    assert(toSqlType(schema).dataType.isInstanceOf[StructType])
+    val serializer = AvroTypeInference.serializerFor(avroClass, schema)
+    val deserializer = AvroTypeInference.deserializerFor(schema)
+    new ExpressionEncoder[T](
+      serializer,
+      deserializer,
+      ClassTag[T](avroClass))
+  }
+
+  def of[T <: IndexedRecord](schema: Schema): ExpressionEncoder[T] = {
+    assert(toSqlType(schema).dataType.isInstanceOf[StructType])
+    val avroClass = Option(ReflectData.get.getClass(schema))
+      .map(_.asSubclass(classOf[SpecificRecord]))
+      .getOrElse(classOf[GenericData.Record])
+    val serializer = AvroTypeInference.serializerFor(avroClass, schema)
+    val deserializer = AvroTypeInference.deserializerFor(schema)
+    new ExpressionEncoder[T](
+      serializer,
+      deserializer,
+      ClassTag[T](avroClass))
+  }
+}
+
+/**
+ * Utilities for providing Avro object serializers and deserializers
+ */
+private object AvroTypeInference {
+
+  /**
+   * Translates an Avro Schema type to a proper SQL DataType. The Java Objects that back data in
+   * generated Generic and Specific records sometimes do not align with those suggested by Avro
+   * ReflectData, so we infer the proper SQL DataType to serialize and deserialize based on
+   * nullability and the wrapping Schema type.
+   */
+  private def inferExternalType(avroSchema: Schema): DataType = {
+    toSqlType(avroSchema) match {
+      // the non-nullable primitive types
+      case SchemaType(BooleanType, false) => BooleanType
+      case SchemaType(IntegerType, false) => IntegerType
+      case SchemaType(LongType, false) =>
+        if (avroSchema.getType == UNION) {
+          ObjectType(classOf[java.lang.Number])
+        } else {
+          LongType
+        }
+      case SchemaType(FloatType, false) => FloatType
+      case SchemaType(DoubleType, false) =>
+        if (avroSchema.getType == UNION) {
+          ObjectType(classOf[java.lang.Number])
+        } else {
+          DoubleType
+        }
+      // the nullable primitive types
+      case SchemaType(BooleanType, true) => ObjectType(classOf[java.lang.Boolean])
+      case SchemaType(IntegerType, true) => ObjectType(classOf[java.lang.Integer])
+      case SchemaType(LongType, true) => ObjectType(classOf[java.lang.Long])
+      case SchemaType(FloatType, true) => ObjectType(classOf[java.lang.Float])
+      case SchemaType(DoubleType, true) => ObjectType(classOf[java.lang.Double])
+      // the binary types
+      case SchemaType(BinaryType, _) =>
 
 Review comment:
   When avroSchema is a Union{null, Fixed}, it results in a java.io.ByteBuffer. Have we considered this case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org