Github user aarondav commented on a diff in the pull request:

    https://github.com/apache/spark/pull/5279#discussion_r27939459
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ReflectionConverters.scala
 ---
    @@ -0,0 +1,263 @@
    +/*
    + * 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.catalyst
    +
    +import java.util.{Map => JavaMap}
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.types._
    +
    +/**
    + * Functions to convert Scala types to Catalyst types and vice versa.
    + */
    +object ReflectionConverters {
    +  // The Predef.Map is scala.collection.immutable.Map.
    +  // Since the map values can be mutable, we explicitly import 
scala.collection.Map at here.
    +  import scala.collection.Map
    +
    +  /**
    +   * Converts Scala objects to catalyst rows / types.
    +   * Note: This is always called after schemaFor has been called.
    +   *       This ordering is important for UDT registration.
    +   */
    +  def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) 
match {
    +    // Check UDT first since UDTs can override other types
    +    case (obj, udt: UserDefinedType[_]) => udt.serialize(obj)
    +    case (o: Option[_], _) => o.map(convertToCatalyst(_, dataType)).orNull
    +    case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, 
arrayType.elementType))
    +    case (s: Array[_], arrayType: ArrayType) =>
    +      s.toSeq.map(convertToCatalyst(_, arrayType.elementType))
    +    case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) =>
    +      convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, 
mapType.valueType)
    +    }
    +    case (jmap: JavaMap[_, _], mapType: MapType) =>
    +      val iter = jmap.entrySet.iterator
    +      var listOfEntries: List[(Any, Any)] = List()
    +      while (iter.hasNext) {
    +        val entry = iter.next()
    +        listOfEntries :+= (convertToCatalyst(entry.getKey, 
mapType.keyType),
    +          convertToCatalyst(entry.getValue, mapType.valueType))
    +      }
    +      listOfEntries.toMap
    +    case (p: Product, structType: StructType) =>
    +      val ar = new Array[Any](structType.size)
    +      val iter = p.productIterator
    +      var idx = 0
    +      while (idx < structType.size) {
    +        ar(idx) = convertToCatalyst(iter.next(), 
structType.fields(idx).dataType)
    +        idx += 1
    +      }
    +      new GenericRowWithSchema(ar, structType)
    +    case (d: BigDecimal, _) => Decimal(d)
    +    case (d: java.math.BigDecimal, _) => Decimal(d)
    +    case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d)
    +    case (r: Row, structType: StructType) =>
    +      val converters = structType.fields.map {
    +        f => (item: Any) => convertToCatalyst(item, f.dataType)
    +      }
    +      convertRowWithConverters(r, structType, converters)
    +    case (other, _) => other
    +  }
    +
    +  /**
    +   * Creates a converter function that will convert Scala objects to the 
specified catalyst type.
    +   */
    +  private[sql] def createCatalystConverter(dataType: DataType): Any => Any 
= {
    +    def extractOption(item: Any): Any = item match {
    +      case s: Some[_] => s.get
    +      case None => null
    +      case other => other
    +    }
    +
    +    dataType match {
    +      // Check UDT first since UDTs can override other types
    +      case udt: UserDefinedType[_] =>
    +        (item) => {
    +          if (item == None) null else udt.serialize(extractOption(item))
    +        }
    +
    +      case arrayType: ArrayType =>
    +        val elementConverter = 
createCatalystConverter(arrayType.elementType)
    +        (item: Any) => {
    +          extractOption(item) match {
    +            case a: Array[_] => a.toSeq.map(elementConverter)
    +            case s: Seq[_] => s.map(elementConverter)
    +            case null => null
    +          }
    +        }
    +
    +      case mapType: MapType =>
    +        val keyConverter = createCatalystConverter(mapType.keyType)
    +        val valueConverter = createCatalystConverter(mapType.valueType)
    +        (item: Any) => {
    +          extractOption(item) match {
    +            case m: Map[_, _] =>
    +              m.map { case (k, v) =>
    +                keyConverter(k) -> valueConverter(v)
    +              }
    +
    +            case jmap: JavaMap[_, _] =>
    +              val iter = jmap.entrySet.iterator
    +              var listOfEntries: List[(Any, Any)] = List()
    +              while (iter.hasNext) {
    +                val entry = iter.next()
    +                listOfEntries :+= (keyConverter(entry.getKey), 
valueConverter(entry.getValue))
    +              }
    +              listOfEntries.toMap
    +
    +            case null => null
    +          }
    +        }
    +
    +      case structType: StructType =>
    +        val converters = new Array[Any => Any](structType.length)
    +        val iter = structType.fields.iterator
    +        var idx = 0
    +        while (iter.hasNext) {
    +          converters(idx) = createCatalystConverter(iter.next().dataType)
    +          idx += 1
    +        }
    +        (item: Any) => {
    +          extractOption(item) match {
    +            case r: Row => convertRowWithConverters(r, structType, 
converters)
    +
    +            case p: Product =>
    +              val ar = new Array[Any](structType.size)
    +              val iter = p.productIterator
    +              var idx = 0
    +              while (idx < structType.size) {
    +                ar(idx) = converters(idx)(iter.next())
    +                idx += 1
    +              }
    +              new GenericRowWithSchema(ar, structType)
    +
    +            case null => null
    +          }
    +        }
    +
    +      case _ =>
    +        (item: Any) => extractOption(item) match {
    +          case d: BigDecimal => Decimal(d)
    +          case d: java.math.BigDecimal => Decimal(d)
    +          case d: java.sql.Date => DateUtils.fromJavaDate(d)
    +          case other => other
    +        }
    +    }
    +  }
    +
    +  /** Converts Catalyst types used internally in rows to standard Scala 
types */
    +  def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) 
match {
    +    // Check UDT first since UDTs can override other types
    +    case (d, udt: UserDefinedType[_]) => udt.deserialize(d)
    +    case (s: Seq[_], arrayType: ArrayType) => s.map(convertToScala(_, 
arrayType.elementType))
    +    case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) =>
    +      convertToScala(k, mapType.keyType) -> convertToScala(v, 
mapType.valueType)
    +    }
    +    case (r: Row, s: StructType) => convertRowToScala(r, s)
    +    case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal
    +    case (i: Int, DateType) => DateUtils.toJavaDate(i)
    +    case (other, _) => other
    +  }
    +
    +  /**
    +   * Creates a converter function that will convert Catalyst types to 
Scala type.
    +   */
    +  private[sql] def createScalaConverter(dataType: DataType): Any => Any = 
dataType match {
    +    // Check UDT first since UDTs can override other types
    +    case udt: UserDefinedType[_] =>
    +      (item: Any) => if (item == null) null else udt.deserialize(item)
    +
    +    case arrayType: ArrayType =>
    +      val elementConverter = createScalaConverter(arrayType.elementType)
    +      (item: Any) => if (item == null) null else 
item.asInstanceOf[Seq[_]].map(elementConverter)
    +
    +    case mapType: MapType =>
    +      val keyConverter = createScalaConverter(mapType.keyType)
    +      val valueConverter = createScalaConverter(mapType.valueType)
    +      (item: Any) => if (item == null) {
    +        null
    +      } else {
    +        item.asInstanceOf[Map[_, _]].map { case (k, v) =>
    +          keyConverter(k) -> valueConverter(v)
    +        }
    +      }
    +
    +    case s: StructType =>
    +      val converters = createScalaConvertersForStruct(s)
    --- End diff --
    
    This implementation is so clear that I would actually just inline it: 
`s.fields.map(f => createScalaConverter(f.dataType))`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to