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

    https://github.com/apache/spark/pull/999#discussion_r13828804
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala 
---
    @@ -0,0 +1,402 @@
    +/*
    + * 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.json
    +
    +import scala.collection.JavaConversions._
    +import scala.math.BigDecimal
    +
    +import com.fasterxml.jackson.databind.ObjectMapper
    +
    +import org.apache.spark.annotation.{DeveloperApi, Experimental}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.plans.logical._
    +import org.apache.spark.sql.catalyst.types._
    +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan}
    +import org.apache.spark.sql.Logging
    +
    +@Experimental
    +private[sql] object JsonRDD extends Logging {
    +
    +  @DeveloperApi
    +  private[sql] def inferSchema(
    +      json: RDD[String],
    +      samplingRatio: Double = 1.0): LogicalPlan = {
    +    require(samplingRatio > 0)
    +    val schemaData = if (samplingRatio > 0.99) json else 
json.sample(false, samplingRatio, 1)
    +
    +    val allKeys = 
parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _)
    +
    +    val baseSchema = createSchema(allKeys)
    +
    +    createLogicalPlan(json, baseSchema)
    +  }
    +
    +  private def createLogicalPlan(
    +      json: RDD[String],
    +      baseSchema: StructType): LogicalPlan = {
    +    val schema = nullTypeToStringType(baseSchema)
    +
    +    SparkLogicalPlan(ExistingRdd(asAttributes(schema), 
parseJson(json).map(asRow(_, schema))))
    +  }
    +
    +  private def createSchema(allKeys: Set[(String, DataType)]): StructType = 
{
    +    // Resolve type conflicts
    +    val resolved = allKeys.groupBy {
    +      case (key, dataType) => key
    +    }.map {
    +      // Now, keys and types are organized in the format of
    +      // key -> Set(type1, type2, ...).
    +      case (key, typeSet) => {
    +        val fieldName = key.substring(1, key.length - 1).split("`.`").toSeq
    +        val dataType = typeSet.map {
    +          case (_, dataType) => dataType
    +        }.reduce((type1: DataType, type2: DataType) => 
getCompatibleType(type1, type2))
    +
    +        (fieldName, dataType)
    +      }
    +    }
    +
    +    def makeStruct(values: Seq[Seq[String]], prefix: Seq[String]): 
StructType = {
    +      val (topLevel, structLike) = values.partition(_.size == 1)
    +      val topLevelFields = topLevel.filter {
    +        name => resolved.get(prefix ++ name).get match {
    +          case ArrayType(StructType(Nil)) => false
    +          case ArrayType(_) => true
    +          case struct: StructType => false
    +          case _ => true
    +        }
    +      }.map {
    +        a => StructField(a.head, resolved.get(prefix ++ a).get, nullable = 
true)
    +      }
    +
    +      val structFields: Seq[StructField] = structLike.groupBy(_(0)).map {
    +        case (name, fields) => {
    +          val nestedFields = fields.map(_.tail)
    +          val structType = makeStruct(nestedFields, prefix :+ name)
    +          val dataType = resolved.get(prefix :+ name).get
    +          dataType match {
    +            case array: ArrayType => Some(StructField(name, 
ArrayType(structType), nullable = true))
    +            case struct: StructType => Some(StructField(name, structType, 
nullable = true))
    +            // dataType is StringType means that we have resolved type 
conflicts involving
    +            // primitive types and complex types. So, the type of name has 
been relaxed to
    +            // StringType. Also, this field should have already been put 
in topLevelFields.
    +            case StringType => None
    +          }
    +        }
    +      }.flatMap(field => field).toSeq
    +
    +      StructType(
    +        (topLevelFields ++ structFields).sortBy {
    +        case StructField(name, _, _) => name
    +      })
    +    }
    +
    +    makeStruct(resolved.keySet.toSeq, Nil)
    +  }
    +
    +  /**
    +   * Returns the most general data type for two given data types.
    +   */
    +  private[json] def getCompatibleType(t1: DataType, t2: DataType): 
DataType = {
    +    // Try and find a promotion rule that contains both types in question.
    +    val applicableConversion = HiveTypeCoercion.allPromotions.find(p => 
p.contains(t1) && p
    +      .contains(t2))
    +
    +    // If found return the widest common type, otherwise None
    +    val returnType = applicableConversion.map(_.filter(t => t == t1 || t 
== t2).last)
    +
    +    if (returnType.isDefined) {
    +      returnType.get
    +    } else {
    +      // t1 or t2 is a StructType, ArrayType, or an unexpected type.
    +      (t1, t2) match {
    +        case (other: DataType, NullType) => other
    +        case (NullType, other: DataType) => other
    +        case (StructType(fields1), StructType(fields2)) => {
    +          val newFields = (fields1 ++ fields2).groupBy(field => 
field.name).map {
    +            case (name, fieldTypes) => {
    +              val dataType = fieldTypes.map(field => 
field.dataType).reduce(
    +                (type1: DataType, type2: DataType) => 
getCompatibleType(type1, type2))
    +              StructField(name, dataType, true)
    +            }
    +          }
    +          StructType(newFields.toSeq.sortBy {
    +            case StructField(name, _, _) => name
    +          })
    +        }
    +        case (ArrayType(elementType1), ArrayType(elementType2)) =>
    +          ArrayType(getCompatibleType(elementType1, elementType2))
    +        // TODO: We should use JsonObjectStringType to mark that values of 
field will be
    +        // strings and every string is a Json object.
    +        case (_, _) => StringType
    +      }
    +    }
    +  }
    +
    +  private def getPrimitiveType(value: Any): DataType = {
    +    value match {
    +      case value: java.lang.String => StringType
    +      case value: java.lang.Integer => IntegerType
    +      case value: java.lang.Long => LongType
    +      // Since we do not have a data type backed by BigInteger,
    +      // when we see a Java BigInteger, we use DecimalType.
    +      case value: java.math.BigInteger => DecimalType
    +      case value: java.lang.Double => DoubleType
    +      case value: java.math.BigDecimal => DecimalType
    +      case value: java.lang.Boolean => BooleanType
    +      case null => NullType
    +      // Unexpected data type.
    +      case _ => StringType
    +    }
    +  }
    +
    +  /**
    +   * Returns the element type of an JSON array. We go through all elements 
of this array
    +   * to detect any possible type conflict. We use [[getCompatibleType]] to 
resolve
    +   * type conflicts. Right now, when the element of an array is another 
array, we
    +   * treat the element as String.
    +   */
    +  private def getTypeOfArray(l: Seq[Any]): ArrayType = {
    +    val elements = l.flatMap(v => Option(v))
    +    if (elements.isEmpty) {
    +      // If this JSON array is empty, we use NullType as a placeholder.
    +      // If this array is not empty in other JSON objects, we can resolve
    +      // the type after we have passed through all JSON objects.
    +      ArrayType(NullType)
    +    } else {
    +      val elementType = elements.map {
    +        e => e match {
    +          case map: Map[_, _] => StructType(Nil)
    +          // We have an array of arrays. If those element arrays do not 
have the same
    +          // element types, we will return ArrayType[StringType].
    +          case seq: Seq[_] =>  getTypeOfArray(seq)
    +          case value => getPrimitiveType(value)
    +        }
    +      }.reduce((type1: DataType, type2: DataType) => 
getCompatibleType(type1, type2))
    +
    +      ArrayType(elementType)
    +    }
    +  }
    +
    +  /**
    +   * Figures out all key names and data types of values from a parsed JSON 
object
    +   * (in the format of Map[Stirng, Any]). When the value of a key is an 
JSON object, we
    +   * only use a placeholder (StructType(Nil)) to mark that it should be a 
struct
    +   * instead of getting all fields of this struct because a field does not 
appear
    +   * in this JSON object can appear in other JSON objects.
    +   */
    +  private def getAllKeysWithValueTypes(m: Map[String, Any]): Set[(String, 
DataType)] = {
    +    m.map{
    +      // Quote the key with backticks to handle cases which have dots
    +      // in the field name.
    +      case (key, dataType) => (s"`$key`", dataType)
    +    }.flatMap {
    +      case (key: String, struct: Map[String, Any]) => {
    +        // The value associted with the key is an JSON object.
    +        getAllKeysWithValueTypes(struct).map {
    +          case (k, dataType) => (s"$key.$k", dataType)
    +        } ++ Set((key, StructType(Nil)))
    +      }
    +      case (key: String, array: List[Any]) => {
    +        // The value associted with the key is an array.
    +        getTypeOfArray(array) match {
    +          case ArrayType(StructType(Nil)) => {
    +            // The elements of this arrays are structs.
    +            array.asInstanceOf[List[Map[String, Any]]].flatMap {
    +              element => getAllKeysWithValueTypes(element)
    +            }.map {
    +              case (k, dataType) => (s"$key.$k", dataType)
    +            } :+ (key, ArrayType(StructType(Nil)))
    +          }
    +          case ArrayType(elementType) => (key, ArrayType(elementType)) :: 
Nil
    +        }
    +      }
    +      case (key: String, value) => (key, getPrimitiveType(value)) :: Nil
    +    }.toSet
    +  }
    +
    +  /**
    +   * Converts a Java Map/List to a Scala Map/List.
    --- End diff --
    
    Is there a reason we can't just `import scala.collection.JavaConversions._`?


---
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.
---

Reply via email to