From f31065f2aac5b5638c3f3a84ee3eb901da816f64 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 6 Jun 2014 12:21:46 -0700 Subject: [PATCH 01/25] A query plan or a SchemaRDD can print out its schema. --- .../spark/sql/catalyst/plans/QueryPlan.scala | 5 ++ .../spark/sql/catalyst/plans/package.scala | 50 ++++++++++++++++++- .../org/apache/spark/sql/SchemaRDDLike.scala | 6 +++ 3 files changed, 60 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 8199a80f5d6bd..819c1c7ed27b5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.plans abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -123,4 +124,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy case other => Nil }.toSeq } + + def printSchema(): Unit = { + println(plans.generateSchemaTreeString(output)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 42bdab42b79ff..dce0ae0751eb3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -17,8 +17,56 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.types.{StructField, DataType, ArrayType, StructType} + /** * A a collection of common abstractions for query plans as well as * a base logical plan representation. */ -package object plans +package object plans { + def generateSchemaTreeString(schema: Seq[Attribute]): String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + schema.foreach { + attribute => { + val name = attribute.name + val dataType = attribute.dataType + dataType match { + case fields: StructType => + builder.append(s"$prefix-- $name: $StructType\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case ArrayType(fields: StructType) => + builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case ArrayType(elementType: DataType) => + builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") + case _ => builder.append(s"$prefix-- $name: $dataType\n") + } + } + } + + builder.toString() + } + + def generateSchemaTreeString( + schema: StructType, + prefix: String, + builder: StringBuilder): StringBuilder = { + schema.fields.foreach { + case StructField(name, fields: StructType, _) => + builder.append(s"$prefix-- $name: $StructType\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case StructField(name, ArrayType(fields: StructType), _) => + builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case StructField(name, ArrayType(elementType: DataType), _) => + builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") + case StructField(name, fieldType: DataType, _) => + builder.append(s"$prefix-- $name: $fieldType\n") + } + + builder + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 3a895e15a4508..2e608099a9250 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -111,4 +111,10 @@ private[sql] trait SchemaRDDLike { @Experimental def saveAsTable(tableName: String): Unit = sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd + + + /** + * Print the schema of this SchemaRDD. + */ + def printSchema = queryExecution.analyzed.printSchema() } From f45583b0f244b6ef9293202333f82bcdc1abcd00 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 6 Jun 2014 13:38:41 -0700 Subject: [PATCH 02/25] Infer the schema of a JSON dataset (a text file with one JSON object per line or a RDD[String] with one JSON object per string) and returns a SchemaRDD. --- project/SparkBuild.scala | 20 +- .../catalyst/analysis/HiveTypeCoercion.scala | 3 + sql/core/pom.xml | 5 + .../org/apache/spark/sql/SQLContext.scala | 36 ++ .../org/apache/spark/sql/json/JsonTable.scala | 339 +++++++++++++++++ .../org/apache/spark/sql/QueryTest.scala | 4 + .../org/apache/spark/sql/json/JsonSuite.scala | 354 ++++++++++++++++++ .../apache/spark/sql/json/TestJsonData.scala | 67 ++++ 8 files changed, 825 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 64c9441d8e3f8..a9fa172a6dc81 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -484,9 +484,23 @@ object SparkBuild extends Build { def sqlCoreSettings = sharedSettings ++ Seq( name := "spark-sql", libraryDependencies ++= Seq( - "com.twitter" % "parquet-column" % parquetVersion, - "com.twitter" % "parquet-hadoop" % parquetVersion - ) + "com.twitter" % "parquet-column" % parquetVersion, + "com.twitter" % "parquet-hadoop" % parquetVersion, + "com.fasterxml.jackson.core" % "jackson-core" % "2.3.2" + ), + initialCommands in console := + """ + |import org.apache.spark.sql.catalyst.analysis._ + |import org.apache.spark.sql.catalyst.dsl._ + |import org.apache.spark.sql.catalyst.errors._ + |import org.apache.spark.sql.catalyst.expressions._ + |import org.apache.spark.sql.catalyst.plans.logical._ + |import org.apache.spark.sql.catalyst.rules._ + |import org.apache.spark.sql.catalyst.types._ + |import org.apache.spark.sql.catalyst.util._ + |import org.apache.spark.sql.execution + |import org.apache.spark.sql.test.TestSQLContext._ + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) // Since we don't include hive in the main assembly this project also acts as an alternative diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 326feea6fee91..8ab9412f9cc76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -108,6 +108,9 @@ trait HiveTypeCoercion { * * Additionally, all types when UNION-ed with strings will be promoted to strings. * Other string conversions are handled by PromoteStrings. + * + * A widening conversion of a value with IntegerType and LongType to FloatType, + * or of a value with LongType to DoubleType, may result in loss of precision. */ object WidenTypes extends Rule[LogicalPlan] { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fb3b190b4ec5a..54a37d2586ffc 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -53,6 +53,11 @@ parquet-hadoop ${parquet.version} + + com.fasterxml.jackson.core + jackson-core + 2.3.2 + org.scalatest scalatest_${scala.binary.version} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 043be58edc91b..6fba3ce016e66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.json._ /** * :: AlphaComponent :: @@ -97,6 +98,41 @@ class SQLContext(@transient val sparkContext: SparkContext) def parquetFile(path: String): SchemaRDD = new SchemaRDD(this, parquet.ParquetRelation(path)) + /** + * Loads a JSON file, returning the result as a [[SchemaRDD]]. + * Right now, we only do eager schema resolution. + */ + def jsonFile( + path: String, + mode: SchemaResolutionMode = EAGER_SCHEMA_RESOLUTION): SchemaRDD = { + logger.info(s"Loads a JSON file $path.") + val json = sparkContext.textFile(path) + jsonRDD(json, mode) + } + + /** + * Loads a RDD[String] storing JSON objects (one object per record), + * returning the result as a [[SchemaRDD]]. + * Right now, we only do eager schema resolution. + */ + def jsonRDD( + json: RDD[String], + mode: SchemaResolutionMode = EAGER_SCHEMA_RESOLUTION): SchemaRDD = { + mode match { + case EAGER_SCHEMA_RESOLUTION => + logger.info(s"Eagerly resolve the schema without sampling.") + val logicalPlan = JsonTable.inferSchema(json) + logicalPlanToSparkQuery(logicalPlan) + case EAGER_SCHEMA_RESOLUTION_WITH_SAMPLING(fraction) => + logger.info(s"Eagerly resolve the schema with sampling " + + s"(sampling fraction: $fraction).") + val logicalPlan = JsonTable.inferSchema(json, Some(fraction)) + logicalPlanToSparkQuery(logicalPlan) + case LAZY_SCHEMA_RESOLUTION => + throw new UnsupportedOperationException("Lazy schema resolution has not been implemented.") + } + } + /** * :: Experimental :: * Creates an empty parquet file with the schema of class `A`, which can be registered as a table. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala new file mode 100644 index 0000000000000..bfcb6315e1541 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -0,0 +1,339 @@ +package org.apache.spark.sql.json + +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, GetField} + +import com.fasterxml.jackson.databind.ObjectMapper + +import scala.collection.JavaConversions._ +import scala.math.BigDecimal +import org.apache.spark.sql.catalyst.expressions.GetField +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.expressions.GetField +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.types.StructField +import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.types.ArrayType +import org.apache.spark.sql.catalyst.expressions.GetField +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.catalyst.expressions.Alias + +sealed trait SchemaResolutionMode + +case object EAGER_SCHEMA_RESOLUTION extends SchemaResolutionMode +case class EAGER_SCHEMA_RESOLUTION_WITH_SAMPLING(val fraction: Double) extends SchemaResolutionMode +case object LAZY_SCHEMA_RESOLUTION extends SchemaResolutionMode + +/** + * :: Experimental :: + * Converts a JSON file to a SparkSQL logical query plan. This implementation is only designed to + * work on JSON files that have mostly uniform schema. The conversion suffers from the following + * limitation: + * - The data is optionally sampled to determine all of the possible fields. Any fields that do + * not appear in this sample will not be included in the final output. + */ +@Experimental +object JsonTable extends Serializable with Logging { + def inferSchema( + json: RDD[String], sampleSchema: Option[Double] = None): LogicalPlan = { + val schemaData = sampleSchema.map(json.sample(false, _, 1)).getOrElse(json) + val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) + // 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) + }.sortBy { + case StructField(name, _, _) => name + } + + 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.sortBy { + case StructField(name, _, _) => name + } + + StructType(topLevelFields ++ structFields) + } + + val schema = makeStruct(resolved.keySet.toSeq, Nil) + + SparkLogicalPlan( + ExistingRdd( + asAttributes(schema), + parseJson(json).map(asRow(_, schema)))) + } + + // numericPrecedence and booleanPrecedence are from WidenTypes. + // A widening conversion of a value with IntegerType and LongType to FloatType, + // or of a value with LongType to DoubleType, may result in loss of precision + // (some of the least significant bits of the value). + val numericPrecedence = + Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) + // Boolean is only wider than Void + val booleanPrecedence = Seq(NullType, BooleanType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil + + /** + * Returns the most general data type for two given data types. + */ + protected def getCompatibleType(t1: DataType, t2: DataType): DataType = { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = 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 + // TODO: Returns the union of fields1 and fields2? + case (StructType(fields1), StructType(fields2)) + if (fields1 == fields2) => StructType(fields1) + case (ArrayType(elementType1), ArrayType(elementType2)) => + ArrayType(getCompatibleType(elementType1, elementType2)) + case (_, _) => StringType + } + } + } + + protected 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 + // We comment out the following line in the development to catch bugs. + // We need to enable this line in future to handle + // 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. + */ + protected 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 a value of a key is an object, we + * only use a placeholder for a struct type (StructType(Nil)) instead of getting + * all fields of this struct because a field does not appear in this JSON object + * can appear in other JSON objects. + */ + protected 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. + * We do not use Jackson's scala module at here because + * DefaultScalaModule in jackson-module-scala will make + * the parsing very slow. + */ + protected def scalafy(obj: Any): Any = obj match { + case map: java.util.Map[String, Object] => + // .map(identity) is used as a workaround of non-serializable Map + // generated by .mapValues. + // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 + map.toMap.mapValues(scalafy).map(identity) + case list: java.util.List[Object] => + list.toList.map(scalafy) + case atom => atom + } + + protected def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { + // According to [Jackson-72: https://jira.codehaus.org/browse/JACKSON-72], + // ObjectMapper will not return BigDecimal when + // "DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS" is disabled + // (see NumberDeserializer.deserialize for the logic). + // But, we do not want to enable this feature because it will use BigDecimal + // for every float number, which will be slow. + // So, right now, we will have Infinity for those BigDecimal number. + // TODO: Support BigDecimal. + json.mapPartitions(iter => { + // When there is a key appearing multiple times (a duplicate key), + // the ObjectMapper will take the last value associated with this duplicate key. + // For example: for {"key": 1, "key":2}, we will get "key"->2. + val mapper = new ObjectMapper() + iter.map(record => mapper.readValue(record, classOf[Object])) + }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) + } + + protected def toLong(value: Any): Long = { + value match { + case value: java.lang.Integer => value.asInstanceOf[Int].asInstanceOf[Long] + case value: java.lang.Long => value.asInstanceOf[Long] + } + } + + protected def toDouble(value: Any): Double = { + value match { + case value: java.lang.Integer => value.asInstanceOf[Int].asInstanceOf[Double] + case value: java.lang.Long => value.asInstanceOf[Long].asInstanceOf[Double] + case value: java.lang.Double => value.asInstanceOf[Double] + } + } + + protected def toDecimal(value: Any): BigDecimal = { + value match { + case value: java.lang.Integer => BigDecimal(value) + case value: java.lang.Long => BigDecimal(value) + case value: java.math.BigInteger => BigDecimal(value) + case value: java.lang.Double => BigDecimal(value) + case value: java.math.BigDecimal => BigDecimal(value) + } + } + + protected def enforceCorrectType(value: Any, desiredType: DataType): Any ={ + if (value == null) { + null + } else { + desiredType match { + case ArrayType(elementType) => + value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) + case StringType => value.toString + case IntegerType => value.asInstanceOf[IntegerType.JvmType] + case LongType => toLong(value) + case DoubleType => toDouble(value) + case DecimalType => toDecimal(value) + case BooleanType => value.asInstanceOf[BooleanType.JvmType] + case NullType => null + } + } + } + + protected def asRow(json: Map[String,Any], schema: StructType): Row = { + val row = new GenericMutableRow(schema.fields.length) + schema.fields.zipWithIndex.foreach { + // StructType + case (StructField(name, fields: StructType, _), i) => + row.update(i, json.get(name).flatMap(v => Option(v)).map( + v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) + + // ArrayType(StructType) + case (StructField(name, ArrayType(structType: StructType), _), i) => + row.update(i, + json.get(name).flatMap(v => Option(v)).map( + v => v.asInstanceOf[Seq[Any]].map( + e => asRow(e.asInstanceOf[Map[String, Any]], structType))).orNull) + + // Other cases + case (StructField(name, dataType, _), i) => + row.update(i, json.get(name).flatMap(v => Option(v)).map( + enforceCorrectType(_, dataType)).getOrElse(null)) + } + + row + } + + protected def asAttributes(struct: StructType): Seq[AttributeReference] = { + struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) + } +} \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index d6072b402a044..3ed5ce239f008 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -45,6 +45,10 @@ class QueryTest extends FunSuite { s""" |Exception thrown while executing query: |${rdd.logicalPlan} + |== Analyzed Plan == + |${rdd.queryExecution.analyzed} + |== Physical Plan == + |${rdd.queryExecution.executedPlan} |== Exception == |$e """.stripMargin) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala new file mode 100644 index 0000000000000..8d4c84d05b963 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -0,0 +1,354 @@ +package org.apache.spark.sql.json + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference, Attribute} +import org.apache.spark.sql.catalyst.plans.generateSchemaTreeString +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.util._ + +class JsonSuite extends QueryTest { + import TestJsonData._ + TestJsonData + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(attributes: Seq[Attribute]) = { + val minId = attributes.map(_.exprId.id).min + attributes.map { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + protected def checkSchema(expected: Seq[Attribute], actual: Seq[Attribute]): Unit = { + val normalizedExpected = normalizeExprIds(expected).toSeq + val normalizedActual = normalizeExprIds(actual).toSeq + if (normalizedExpected != normalizedActual) { + fail( + s""" + |=== FAIL: Schemas do not match === + |${sideBySide( + s"== Expected Schema ==\n" + + generateSchemaTreeString(normalizedExpected), + s"== Actual Schema ==\n" + + generateSchemaTreeString(normalizedActual)).mkString("\n")} + """.stripMargin) + } + } + + test("Primitive field and type inferring") { + val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) + + val expectedSchema = + AttributeReference("bigInteger", DecimalType, true)() :: + AttributeReference("boolean", BooleanType, true)() :: + AttributeReference("double", DoubleType, true)() :: + AttributeReference("integer", IntegerType, true)() :: + AttributeReference("long", LongType, true)() :: + AttributeReference("null", NullType, true)() :: + AttributeReference("string", StringType, true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + } + + test("Complex field and type inferring") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType) + + val expectedSchema = + AttributeReference("arrayOfArray1", ArrayType(ArrayType(StringType)), true)() :: + AttributeReference("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true)() :: + AttributeReference("arrayOfBigInteger", ArrayType(DecimalType), true)() :: + AttributeReference("arrayOfBoolean", ArrayType(BooleanType), true)() :: + AttributeReference("arrayOfDouble", ArrayType(DoubleType), true)() :: + AttributeReference("arrayOfInteger", ArrayType(IntegerType), true)() :: + AttributeReference("arrayOfLong", ArrayType(LongType), true)() :: + AttributeReference("arrayOfNull", ArrayType(NullType), true)() :: + AttributeReference("arrayOfString", ArrayType(StringType), true)() :: + AttributeReference("arrayOfStruct", ArrayType( + StructType(StructField("field1", BooleanType, true) :: + StructField("field2", StringType, true) :: Nil)), true)() :: + AttributeReference("struct", StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", DecimalType, true) :: Nil), true)() :: + AttributeReference("structWithArrayFields", StructType( + StructField("field1", ArrayType(IntegerType), true) :: + StructField("field2", ArrayType(StringType), true) :: Nil), true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + + // Access elements of a primitive array. + checkAnswer( + sql("select arrayOfString[0], arrayOfString[1], arrayOfString[2] from jsonTable"), + ("str1", "str2", null) :: Nil + ) + + // Access an array of null values. + checkAnswer( + sql("select arrayOfNull from jsonTable"), + Seq(Seq(null, null, null, null)) :: Nil + ) + + // Access elements of a BigInteger array (we use DecimalType internally). + checkAnswer( + sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from jsonTable"), + (BigDecimal("922337203685477580700"), BigDecimal("-922337203685477580800"), null) :: Nil + ) + + // Access elements of an array of arrays. + checkAnswer( + sql("select arrayOfArray1[0], arrayOfArray1[1] from jsonTable"), + (Seq("1", "2", "3"), Seq("str1", "str2")) :: Nil + ) + + // Access elements of an array of arrays. + checkAnswer( + sql("select arrayOfArray2[0], arrayOfArray2[1] from jsonTable"), + (Seq(1.0, 2.0, 3.0), Seq(1.1, 2.1, 3.1)) :: Nil + ) + + // Access elements of an array inside a filed with the type of ArrayType(ArrayType). + checkAnswer( + sql("select arrayOfArray1[1][1], arrayOfArray2[1][1] from jsonTable"), + ("str2", 2.1) :: Nil + ) + + // Access elements of an array of structs. + checkAnswer( + sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2] from jsonTable"), + (true :: "str1" :: Nil, false :: null :: Nil, null) :: Nil + ) + + /* + // Right now, "field1" and "field2" are treated as aliases. We should fix it. + // TODO: Re-enable the following test. + checkAnswer( + sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), + (true, "str1") :: Nil + ) + */ + + /* + // Right now, the analyzer cannot resolve arrayOfStruct.field1 and arrayOfStruct.field2. + // TODO: Re-enable the following test. + // Getting all values of a specific field from an array of structs. + checkAnswer( + sql("select arrayOfStruct.field1, arrayOfStruct.field2 from jsonTable"), + (Seq(true, false), Seq("str1", null)) :: Nil + ) + */ + + // Access a struct and fields inside of it. + checkAnswer( + sql("select struct, struct.field1, struct.field2 from jsonTable"), + ( + Seq(true, BigDecimal("92233720368547758070")), + true, + BigDecimal("92233720368547758070")) :: Nil + ) + + // Access an array field of a struct. + checkAnswer( + sql("select structWithArrayFields.field1, structWithArrayFields.field2 from jsonTable"), + (Seq(4, 5, 6), Seq("str1", "str2")) :: Nil + ) + + // Access elements of an array field of a struct. + checkAnswer( + sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from jsonTable"), + (5, null) :: Nil + ) + } + + test("Type conflict in primitive field values") { + val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) + + val expectedSchema = + AttributeReference("num_bool", StringType, true)() :: + AttributeReference("num_num_1", LongType, true)() :: + AttributeReference("num_num_2", DecimalType, true)() :: + AttributeReference("num_num_3", DoubleType, true)() :: + AttributeReference("num_str", StringType, true)() :: + AttributeReference("str_bool", StringType, true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + ("true", 11L, null, 1.1, "13.1", "str1") :: + ("12", null, BigDecimal("21474836470.9"), null, null, "true") :: + ("false", 21474836470L, BigDecimal("92233720368547758070"), 100, "str1", "false") :: + (null, 21474836570L, BigDecimal(1.1), 21474836470L, "92233720368547758070", null) :: Nil + ) + + // Number and Boolean conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_bool - 10 from jsonTable where num_bool > 11"), + 2 + ) + + /* + // Right now, the analyzer does not insert a Cast for num_bool. + // Number and Boolean conflict: resolve the type as boolean in this query. + // TODO: Re-enable this test + checkAnswer( + sql("select num_bool from jsonTable where NOT num_bool"), + false + ) + */ + + /* + // Right now, the analyzer does not know that num_bool should be treated as a boolean. + // Number and Boolean conflict: resolve the type as boolean in this query. + // TODO: Re-enable this test + checkAnswer( + sql("select num_bool from jsonTable where num_bool"), + true + ) + */ + + // Widening to LongType + checkAnswer( + sql("select num_num_1 - 100 from jsonTable where num_num_1 > 11"), + Seq(21474836370L) :: Seq(21474836470L) :: Nil + ) + + checkAnswer( + sql("select num_num_1 - 100 from jsonTable where num_num_1 > 10"), + Seq(-89) :: Seq(21474836370L) :: Seq(21474836470L) :: Nil + ) + + // Widening to DecimalType + checkAnswer( + sql("select num_num_2 + 1.2 from jsonTable where num_num_2 > 1.1"), + Seq(BigDecimal("21474836472.1")) :: Seq(BigDecimal("92233720368547758071.2")) :: Nil + ) + + // Widening to DoubleType + checkAnswer( + sql("select num_num_3 + 1.2 from jsonTable where num_num_3 > 1.1"), + Seq(101.2) :: Seq(21474836471.2) :: Nil + ) + + /* + // Right now, we have a parsing error. + // Number and String conflict: resolve the type as number in this query. + // TODO: Re-enable this test + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), + BigDecimal("92233720368547758061.2") + ) + */ + + // Number and String conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 14"), + 92233720368547758071.2 + // Seq(14.3) :: Seq(92233720368547758071.2) :: Nil + ) + + /* + // The following test will fail. The type of num_str is StringType. + // So, to evaluate num_str + 1.2, we first need to use Cast to convert the type. + // In our test data, one value of num_str is 13.1. + // The result of (CAST(num_str#65:4, DoubleType) + 1.2) for this value is 14.299999999999999, + // which is not 14.3. + // Number and String conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 13"), + Seq(14.3) :: Seq(92233720368547758071.2) :: Nil + ) + */ + + // String and Boolean conflict: resolve the type as string. + checkAnswer( + sql("select * from jsonTable where str_bool = 'str1'"), + ("true", 11L, null, 1.1, "13.1", "str1") :: Nil + ) + + // TODO: Need to test converting str_bool to boolean values. + // Right now, it has the same issues with tests above on num_bool. + } + + test("Type conflict in complex field values") { + val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) + + val expectedSchema = + AttributeReference("array", ArrayType(IntegerType), true)() :: + AttributeReference("num_struct", StringType, true)() :: + AttributeReference("str_array", StringType, true)() :: + AttributeReference("struct_array", StringType, true)() :: + AttributeReference("struct", StructType( + StructField("field", StringType, true) :: Nil), true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + (Seq(), "11", "List(1, 2, 3)", "List()", Seq(null)) :: + (null, "Map(field -> false)", null, "Map()", null) :: + (Seq(4, 5, 6), null, "str", "List(7, 8, 9)", Seq(null)) :: + (Seq(7), "Map()","List(str1, str2, 33)", "Map(field -> true)", Seq("str")) :: Nil + ) + } + + test("Type conflict in array elements") { + val jsonSchemaRDD = jsonRDD(arrayElementTypeConflict) + + val expectedSchema = + AttributeReference("array", ArrayType(StringType), true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + Seq(Seq("1", "1.1", "true", null, "List()", "Map()", "List(2, 3, 4)", + "Map(field -> str)")) :: Nil + ) + + // Treat an element as a number. + checkAnswer( + sql("select array[0] + 1 from jsonTable"), + 2 + ) + } + + test("Handle missing fields") { + val jsonSchemaRDD = jsonRDD(missingFields) + + val expectedSchema = + AttributeReference("a", BooleanType, true)() :: + AttributeReference("b", IntegerType, true)() :: + AttributeReference("c", ArrayType(IntegerType), true)() :: + AttributeReference("e", StringType, true)() :: + AttributeReference("d", StructType( + StructField("field", BooleanType, true) :: Nil), true)() :: Nil + + checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + + jsonSchemaRDD.registerAsTable("jsonTable") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala new file mode 100644 index 0000000000000..bbad370d9a610 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -0,0 +1,67 @@ +package org.apache.spark.sql.json + +import org.apache.spark.sql.test.TestSQLContext + +object TestJsonData { + + val primitiveFieldAndType = + TestSQLContext.sparkContext.parallelize( + """{"string":"this is a simple string.", """ + + """ "integer":10, """ + + """ "long":21474836470, """ + + """ "bigInteger":92233720368547758070, """ + + """ "double":1.7976931348623157E308, """ + + """ "boolean":true, """ + + """ "null":null """ + + """ } """ :: Nil) + + val complexFieldAndType = + TestSQLContext.sparkContext.parallelize( + """{"struct":{"field1": true, "field2": 92233720368547758070}, """ + + """ "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, """ + + """ "arrayOfString":["str1", "str2"], """ + + """ "arrayOfInteger":[1, 2147483647, -2147483648], """ + + """ "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], """ + + """ "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], """ + + """ "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], """ + + """ "arrayOfBoolean":[true, false, true], """ + + """ "arrayOfNull":[null, null, null, null], """ + + """ "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}], """ + + """ "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], """ + + """ "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] """ + + """ } """ :: Nil) + + val primitiveFieldValueTypeConflict = + TestSQLContext.sparkContext.parallelize( + """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, """ + + """ "num_bool":true, "num_str":13.1, "str_bool":"str1"}""" :: + """{"num_num_1":null, "num_num_2":21474836470.9, "num_num_3": null, """ + + """ "num_bool":12, "num_str":null, "str_bool":true}""" :: + """{"num_num_1":21474836470, "num_num_2":92233720368547758070, "num_num_3": 100, """ + + """ "num_bool":false, "num_str":"str1", "str_bool":false}""" :: + """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470, """ + + """ "num_bool":null, "num_str":92233720368547758070, "str_bool":null}""" :: Nil) + + val complexFieldValueTypeConflict = + TestSQLContext.sparkContext.parallelize( + """{"num_struct":11, "str_array":[1, 2, 3], """ + + """ "array":[], "struct_array":[], "struct": {}}""" :: + """{"num_struct":{"field":false}, "str_array":null, """ + + """ "array":null, "struct_array":{}, "struct": null}""" :: + """{"num_struct":null, "str_array":"str", """ + + """ "array":[4, 5, 6], "struct_array":[7, 8, 9], "struct": {"field":null}}""" :: + """{"num_struct":{}, "str_array":["str1", "str2", 33], """ + + """ "array":[7], "struct_array":{"field": true}, "struct": {"field": "str"}}""" :: Nil) + + val arrayElementTypeConflict = + TestSQLContext.sparkContext.parallelize( + """{"array": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}]}""" :: Nil) + + val missingFields = + TestSQLContext.sparkContext.parallelize( + """{"a":true}""" :: + """{"b":22}""" :: + """{"c":[33, 44]}""" :: + """{"d":{"field":true}}""" :: + """{"e":"str"}""" :: Nil) +} From 05764062719738acea01c95363aae65368e140a2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 6 Jun 2014 13:51:51 -0700 Subject: [PATCH 03/25] Add Apache license header. --- .../org/apache/spark/sql/json/JsonTable.scala | 17 +++++++++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 17 +++++++++++++++++ .../apache/spark/sql/json/TestJsonData.scala | 17 +++++++++++++++++ 3 files changed, 51 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index bfcb6315e1541..8ac1f4b02c3e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -1,3 +1,20 @@ +/* + * 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 org.apache.spark.annotation.Experimental diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 8d4c84d05b963..2889cf349e164 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -1,3 +1,20 @@ +/* + * 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 org.apache.spark.sql.QueryTest diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index bbad370d9a610..9755c58a85603 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -1,3 +1,20 @@ +/* + * 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 org.apache.spark.sql.test.TestSQLContext From f3ce176ef60aeeaba799c4d196975903210dcb6d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 6 Jun 2014 14:22:42 -0700 Subject: [PATCH 04/25] After type conflict resolution, if a NullType is found, StringType is used. --- .../scala/org/apache/spark/sql/json/JsonTable.scala | 12 ++++++++++-- .../scala/org/apache/spark/sql/json/JsonSuite.scala | 4 ++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 8ac1f4b02c3e8..10e29dd42f6d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -67,6 +67,7 @@ object JsonTable extends Serializable with Logging { json: RDD[String], sampleSchema: Option[Double] = None): LogicalPlan = { val schemaData = sampleSchema.map(json.sample(false, _, 1)).getOrElse(json) val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) + // Resolve type conflicts val resolved = allKeys.groupBy { case (key, dataType) => key @@ -79,7 +80,14 @@ object JsonTable extends Serializable with Logging { case (_, dataType) => dataType }.reduce((type1: DataType, type2: DataType) => getCompatibleType(type1, type2)) - (fieldName, dataType) + // Finally, we replace all NullType to StringType. We do not need to take care + // StructType because all fields with a StructType are represented by a placeholder + // StructType(Nil). + dataType match { + case NullType => (fieldName, StringType) + case ArrayType(NullType) => (fieldName, ArrayType(StringType)) + case other => (fieldName, other) + } } } @@ -353,4 +361,4 @@ object JsonTable extends Serializable with Logging { protected def asAttributes(struct: StructType): Seq[AttributeReference] = { struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 2889cf349e164..4e4216386bb0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -65,7 +65,7 @@ class JsonSuite extends QueryTest { AttributeReference("double", DoubleType, true)() :: AttributeReference("integer", IntegerType, true)() :: AttributeReference("long", LongType, true)() :: - AttributeReference("null", NullType, true)() :: + AttributeReference("null", StringType, true)() :: AttributeReference("string", StringType, true)() :: Nil checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) @@ -95,7 +95,7 @@ class JsonSuite extends QueryTest { AttributeReference("arrayOfDouble", ArrayType(DoubleType), true)() :: AttributeReference("arrayOfInteger", ArrayType(IntegerType), true)() :: AttributeReference("arrayOfLong", ArrayType(LongType), true)() :: - AttributeReference("arrayOfNull", ArrayType(NullType), true)() :: + AttributeReference("arrayOfNull", ArrayType(StringType), true)() :: AttributeReference("arrayOfString", ArrayType(StringType), true)() :: AttributeReference("arrayOfStruct", ArrayType( StructType(StructField("field1", BooleanType, true) :: From a2313a68e0f5fa67c6f2873019d000c7710dc1d0 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 9 Jun 2014 17:18:23 -0700 Subject: [PATCH 05/25] Address PR comments. --- .../catalyst/analysis/HiveTypeCoercion.scala | 26 +++--- .../spark/sql/catalyst/plans/package.scala | 6 +- .../org/apache/spark/sql/SQLContext.scala | 12 ++- .../org/apache/spark/sql/SchemaRDDLike.scala | 3 +- .../org/apache/spark/sql/json/JsonTable.scala | 82 ++++++------------- .../org/apache/spark/sql/json/JsonSuite.scala | 1 + 6 files changed, 50 insertions(+), 80 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 8ab9412f9cc76..3e38a04882902 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -22,6 +22,16 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types._ +object HiveTypeCoercion { + // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. + // The conversion for integral and floating point types have a linear widening hierarchy: + val numericPrecedence = + Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) + // Boolean is only wider than Void + val booleanPrecedence = Seq(NullType, BooleanType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil +} + /** * A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that * participate in operations into compatible ones. Most of these rules are based on Hive semantics, @@ -109,21 +119,17 @@ trait HiveTypeCoercion { * Additionally, all types when UNION-ed with strings will be promoted to strings. * Other string conversions are handled by PromoteStrings. * - * A widening conversion of a value with IntegerType and LongType to FloatType, - * or of a value with LongType to DoubleType, may result in loss of precision. + * Widening types might result in loss of precision in the following cases: + * - IntegerType to FloatType + * - LongType to FloatType + * - LongType to DoubleType */ object WidenTypes extends Rule[LogicalPlan] { - // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. - // The conversion for integral and floating point types have a linear widening hierarchy: - val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - // Boolean is only wider than Void - val booleanPrecedence = Seq(NullType, BooleanType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { // Try and find a promotion rule that contains both types in question. - val applicableConversion = allPromotions.find(p => p.contains(t1) && p.contains(t2)) + val applicableConversion = HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p + .contains(t2)) // If found return the widest common type, otherwise None applicableConversion.map(_.filter(t => t == t1 || t == t2).last) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index dce0ae0751eb3..1c9caf33156b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.types.{StructField, DataType, ArrayType, StructType} +import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} /** * A a collection of common abstractions for query plans as well as @@ -29,8 +29,7 @@ package object plans { val builder = new StringBuilder builder.append("root\n") val prefix = " |" - schema.foreach { - attribute => { + schema.foreach { attribute => val name = attribute.name val dataType = attribute.dataType dataType match { @@ -44,7 +43,6 @@ package object plans { builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") case _ => builder.append(s"$prefix-- $name: $dataType\n") } - } } builder.toString() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 6fba3ce016e66..57f285bf8ca5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -104,7 +104,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonFile( path: String, - mode: SchemaResolutionMode = EAGER_SCHEMA_RESOLUTION): SchemaRDD = { + mode: SchemaResolutionMode = EagerSchemaResolution): SchemaRDD = { logger.info(s"Loads a JSON file $path.") val json = sparkContext.textFile(path) jsonRDD(json, mode) @@ -117,19 +117,17 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonRDD( json: RDD[String], - mode: SchemaResolutionMode = EAGER_SCHEMA_RESOLUTION): SchemaRDD = { + mode: SchemaResolutionMode = EagerSchemaResolution): SchemaRDD = { mode match { - case EAGER_SCHEMA_RESOLUTION => + case EagerSchemaResolution => logger.info(s"Eagerly resolve the schema without sampling.") val logicalPlan = JsonTable.inferSchema(json) logicalPlanToSparkQuery(logicalPlan) - case EAGER_SCHEMA_RESOLUTION_WITH_SAMPLING(fraction) => + case EagerSchemaResolutionWithSampling(fraction) => logger.info(s"Eagerly resolve the schema with sampling " + s"(sampling fraction: $fraction).") - val logicalPlan = JsonTable.inferSchema(json, Some(fraction)) + val logicalPlan = JsonTable.inferSchema(json, fraction) logicalPlanToSparkQuery(logicalPlan) - case LAZY_SCHEMA_RESOLUTION => - throw new UnsupportedOperationException("Lazy schema resolution has not been implemented.") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 2e608099a9250..25e6a10f6e2ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -112,9 +112,8 @@ private[sql] trait SchemaRDDLike { def saveAsTable(tableName: String): Unit = sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd - /** * Print the schema of this SchemaRDD. */ - def printSchema = queryExecution.analyzed.printSchema() + def printSchema: Unit = queryExecution.analyzed.printSchema() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 10e29dd42f6d9..5fb6f0169fb7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -17,41 +17,24 @@ 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.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.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.SchemaRDD +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.sql.Logging -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, GetField} - -import com.fasterxml.jackson.databind.ObjectMapper - -import scala.collection.JavaConversions._ -import scala.math.BigDecimal -import org.apache.spark.sql.catalyst.expressions.GetField -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.execution.SparkLogicalPlan -import org.apache.spark.sql.catalyst.expressions.Alias -import org.apache.spark.sql.catalyst.expressions.GetField -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.execution.SparkLogicalPlan -import org.apache.spark.sql.catalyst.expressions.Alias -import org.apache.spark.sql.catalyst.types.StructField -import org.apache.spark.sql.catalyst.types.StructType -import org.apache.spark.sql.catalyst.types.ArrayType -import org.apache.spark.sql.catalyst.expressions.GetField -import org.apache.spark.sql.catalyst.expressions.AttributeReference -import org.apache.spark.sql.execution.SparkLogicalPlan -import org.apache.spark.sql.catalyst.expressions.Alias sealed trait SchemaResolutionMode -case object EAGER_SCHEMA_RESOLUTION extends SchemaResolutionMode -case class EAGER_SCHEMA_RESOLUTION_WITH_SAMPLING(val fraction: Double) extends SchemaResolutionMode -case object LAZY_SCHEMA_RESOLUTION extends SchemaResolutionMode +case object EagerSchemaResolution extends SchemaResolutionMode +case class EagerSchemaResolutionWithSampling(fraction: Double) extends SchemaResolutionMode /** * :: Experimental :: @@ -62,10 +45,9 @@ case object LAZY_SCHEMA_RESOLUTION extends SchemaResolutionMode * not appear in this sample will not be included in the final output. */ @Experimental -object JsonTable extends Serializable with Logging { - def inferSchema( - json: RDD[String], sampleSchema: Option[Double] = None): LogicalPlan = { - val schemaData = sampleSchema.map(json.sample(false, _, 1)).getOrElse(json) +object JsonTable extends Logging { + def inferSchema(json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { + val schemaData = if (samplingRatio > 0.99) json.sample(false, samplingRatio, 1) else json val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) // Resolve type conflicts @@ -129,28 +111,16 @@ object JsonTable extends Serializable with Logging { val schema = makeStruct(resolved.keySet.toSeq, Nil) - SparkLogicalPlan( - ExistingRdd( - asAttributes(schema), - parseJson(json).map(asRow(_, schema)))) + SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) } - // numericPrecedence and booleanPrecedence are from WidenTypes. - // A widening conversion of a value with IntegerType and LongType to FloatType, - // or of a value with LongType to DoubleType, may result in loss of precision - // (some of the least significant bits of the value). - val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - // Boolean is only wider than Void - val booleanPrecedence = Seq(NullType, BooleanType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil - /** * Returns the most general data type for two given data types. */ protected def getCompatibleType(t1: DataType, t2: DataType): DataType = { // Try and find a promotion rule that contains both types in question. - val applicableConversion = allPromotions.find(p => p.contains(t1) && p.contains(t2)) + 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) @@ -184,10 +154,8 @@ object JsonTable extends Serializable with Logging { case value: java.math.BigDecimal => DecimalType case value: java.lang.Boolean => BooleanType case null => NullType - // We comment out the following line in the development to catch bugs. - // We need to enable this line in future to handle - // unexpected data type. - // case _ => StringType + // Unexpected data type. + case _ => StringType } } @@ -221,10 +189,10 @@ object JsonTable extends Serializable with Logging { /** * Figures out all key names and data types of values from a parsed JSON object - * (in the format of Map[Stirng, Any]). When a value of a key is an object, we - * only use a placeholder for a struct type (StructType(Nil)) instead of getting - * all fields of this struct because a field does not appear in this JSON object - * can appear in other JSON objects. + * (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. */ protected def getAllKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { m.map{ @@ -293,15 +261,15 @@ object JsonTable extends Serializable with Logging { protected def toLong(value: Any): Long = { value match { - case value: java.lang.Integer => value.asInstanceOf[Int].asInstanceOf[Long] + case value: java.lang.Integer => value.asInstanceOf[Int].toLong case value: java.lang.Long => value.asInstanceOf[Long] } } protected def toDouble(value: Any): Double = { value match { - case value: java.lang.Integer => value.asInstanceOf[Int].asInstanceOf[Double] - case value: java.lang.Long => value.asInstanceOf[Long].asInstanceOf[Double] + case value: java.lang.Integer => value.asInstanceOf[Int].toDouble + case value: java.lang.Long => value.asInstanceOf[Long].toDouble case value: java.lang.Double => value.asInstanceOf[Double] } } @@ -316,7 +284,7 @@ object JsonTable extends Serializable with Logging { } } - protected def enforceCorrectType(value: Any, desiredType: DataType): Any ={ + protected[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { null } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 4e4216386bb0b..1cb945db2507a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference, At import org.apache.spark.sql.catalyst.plans.generateSchemaTreeString import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.json.JsonTable._ class JsonSuite extends QueryTest { import TestJsonData._ From 038752362d9281870b589a8513d1334c18c950e6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 10 Jun 2014 09:28:58 -0700 Subject: [PATCH 06/25] Address PR comments. --- project/SparkBuild.scala | 2 +- sql/catalyst/pom.xml | 28 ++++++ .../spark/sql/catalyst/plans/QueryPlan.scala | 8 +- .../spark/sql/catalyst/plans/package.scala | 1 + .../catalyst/optimizer/OptimizerTest.scala | 33 +------ .../spark/sql/catalyst/plans/PlanTest.scala | 37 ++++++++ sql/core/pom.xml | 7 ++ .../org/apache/spark/sql/QueryTest.scala | 4 +- .../org/apache/spark/sql/json/JsonSuite.scala | 93 +++++++++++-------- .../apache/spark/sql/json/TestJsonData.scala | 74 +++++++-------- 10 files changed, 171 insertions(+), 116 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d52dd163f0e46..39c4ddb01fb97 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -72,7 +72,7 @@ object SparkBuild extends Build { lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) dependsOn(core) - lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) + lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core) dependsOn(catalyst % "compile->compile;test->test") lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6c78c34486010..01d7b569080ea 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -66,6 +66,34 @@ org.scalatest scalatest-maven-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + + + + test-jar + + + + test-jar-on-compile + compile + + test-jar + + + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 819c1c7ed27b5..a7cb4ec87784e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -125,7 +125,9 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - def printSchema(): Unit = { - println(plans.generateSchemaTreeString(output)) - } + /** Returns the output schema in the tree format. */ + def getSchemaTreeString(): String = plans.generateSchemaTreeString(output) + + /** Prints out the schema in the tree format */ + def printSchema(): Unit = println(getSchemaTreeString) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 1c9caf33156b6..9a1571a682cb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, St * a base logical plan representation. */ package object plans { + def generateSchemaTreeString(schema: Seq[Attribute]): String = { val builder = new StringBuilder builder.append("root\n") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala index 89982d5cd8d74..316b5e3bc5d41 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala @@ -17,39 +17,10 @@ package org.apache.spark.sql.catalyst.optimizer -import org.scalatest.FunSuite - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.plans.PlanTest /** * Provides helper methods for comparing plans produced by optimization rules with the expected * result */ -class OptimizerTest extends FunSuite { - - /** - * Since attribute references are given globally unique ids during analysis, - * we must normalize them to check if two different queries are identical. - */ - protected def normalizeExprIds(plan: LogicalPlan) = { - val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min - plan transformAllExpressions { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) - } - } - - /** Fails the test if the two plans do not match */ - protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { - val normalized1 = normalizeExprIds(plan1) - val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) - fail( - s""" - |== FAIL: Plans do not match === - |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } -} +class OptimizerTest extends PlanTest diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala new file mode 100644 index 0000000000000..bc808a138efb8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -0,0 +1,37 @@ +package org.apache.spark.sql.catalyst.plans + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ + +/** + * Provides helper methods for comparing plans. + */ +class PlanTest extends FunSuite { + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min + plan transformAllExpressions { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizeExprIds(plan1) + val normalized2 = normalizeExprIds(plan2) + if (normalized1 != normalized2) + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } +} diff --git a/sql/core/pom.xml b/sql/core/pom.xml index bdbf25a2687a0..f655f23e7961a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -43,6 +43,13 @@ spark-catalyst_${scala.binary.version} ${project.version} + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + com.twitter parquet-column diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index d7f6abaf5d381..ef84ead2e6e8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql -import org.scalatest.FunSuite - import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ -class QueryTest extends FunSuite { +class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1cb945db2507a..2ed95b59e2063 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,46 +17,19 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.TestSQLContext._ -import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference, Attribute} -import org.apache.spark.sql.catalyst.plans.generateSchemaTreeString +import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.json.JsonTable._ +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.json.JsonTable.enforceCorrectType +import org.apache.spark.sql.QueryTest + +protected case class Schema(output: Seq[Attribute]) extends LeafNode class JsonSuite extends QueryTest { import TestJsonData._ TestJsonData - /** - * Since attribute references are given globally unique ids during analysis, - * we must normalize them to check if two different queries are identical. - */ - protected def normalizeExprIds(attributes: Seq[Attribute]) = { - val minId = attributes.map(_.exprId.id).min - attributes.map { - case a: AttributeReference => - AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) - } - } - - protected def checkSchema(expected: Seq[Attribute], actual: Seq[Attribute]): Unit = { - val normalizedExpected = normalizeExprIds(expected).toSeq - val normalizedActual = normalizeExprIds(actual).toSeq - if (normalizedExpected != normalizedActual) { - fail( - s""" - |=== FAIL: Schemas do not match === - |${sideBySide( - s"== Expected Schema ==\n" + - generateSchemaTreeString(normalizedExpected), - s"== Actual Schema ==\n" + - generateSchemaTreeString(normalizedActual)).mkString("\n")} - """.stripMargin) - } - } - test("Primitive field and type inferring") { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) @@ -69,7 +42,7 @@ class JsonSuite extends QueryTest { AttributeReference("null", StringType, true)() :: AttributeReference("string", StringType, true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") @@ -108,7 +81,7 @@ class JsonSuite extends QueryTest { StructField("field1", ArrayType(IntegerType), true) :: StructField("field2", ArrayType(StringType), true) :: Nil), true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") @@ -206,7 +179,7 @@ class JsonSuite extends QueryTest { AttributeReference("num_str", StringType, true)() :: AttributeReference("str_bool", StringType, true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") @@ -225,7 +198,7 @@ class JsonSuite extends QueryTest { ) /* - // Right now, the analyzer does not insert a Cast for num_bool. + // Right now, the analyzer does not promote strings in a boolean expreesion. // Number and Boolean conflict: resolve the type as boolean in this query. // TODO: Re-enable this test checkAnswer( @@ -275,6 +248,19 @@ class JsonSuite extends QueryTest { sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), BigDecimal("92233720368547758061.2") ) + + // The plan of the following DSL is + // Project [(CAST(num_str#65:4, DoubleType) + 1.2) AS num#78] + // Filter (CAST(CAST(num_str#65:4, DoubleType), DecimalType) > 92233720368547758060) + // ExistingRdd [num_bool#61,num_num_1#62L,num_num_2#63,num_num_3#64,num_str#65,str_bool#66] + // We should directly cast num_str to DecimalType and also need to do the right type promotion + // in the Project. + checkAnswer( + jsonSchemaRDD. + where('num_str > BigDecimal("92233720368547758060")). + select('num_str + 1.2 as Symbol("num")), + BigDecimal("92233720368547758061.2") + ) */ // Number and String conflict: resolve the type as number in this query. @@ -318,7 +304,7 @@ class JsonSuite extends QueryTest { AttributeReference("struct", StructType( StructField("field", StringType, true) :: Nil), true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") @@ -337,7 +323,7 @@ class JsonSuite extends QueryTest { val expectedSchema = AttributeReference("array", ArrayType(StringType), true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") @@ -365,8 +351,33 @@ class JsonSuite extends QueryTest { AttributeReference("d", StructType( StructField("field", BooleanType, true) :: Nil), true)() :: Nil - checkSchema(expectedSchema, jsonSchemaRDD.logicalPlan.output) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") } + + test("Type promotion") { + def checkTypePromotion(expected: Any, actual: Any) { + assert(expected.getClass == actual.getClass, + s"Failed to promote ${actual.getClass} to ${expected.getClass}.") + assert(expected == actual, + s"Promoted value ${actual}(${actual.getClass}) does not equal the expected value " + + s"${expected}(${expected.getClass}).") + } + + val intNumber: Int = 2147483647 + checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType)) + checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType)) + checkTypePromotion(intNumber.toDouble, enforceCorrectType(intNumber, DoubleType)) + checkTypePromotion(BigDecimal(intNumber), enforceCorrectType(intNumber, DecimalType)) + + val longNumber: Long = 9223372036854775807L + checkTypePromotion(longNumber, enforceCorrectType(longNumber, LongType)) + checkTypePromotion(longNumber.toDouble, enforceCorrectType(longNumber, DoubleType)) + checkTypePromotion(BigDecimal(longNumber), enforceCorrectType(longNumber, DecimalType)) + + val doubleNumber: Double = 1.7976931348623157E308d + checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) + checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 9755c58a85603..fe4c95471f8eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -23,52 +23,52 @@ object TestJsonData { val primitiveFieldAndType = TestSQLContext.sparkContext.parallelize( - """{"string":"this is a simple string.", """ + - """ "integer":10, """ + - """ "long":21474836470, """ + - """ "bigInteger":92233720368547758070, """ + - """ "double":1.7976931348623157E308, """ + - """ "boolean":true, """ + - """ "null":null """ + - """ } """ :: Nil) + """{"string":"this is a simple string.", + "integer":10, + "long":21474836470, + "bigInteger":92233720368547758070, + "double":1.7976931348623157E308, + "boolean":true, + "null":null + }""" :: Nil) val complexFieldAndType = TestSQLContext.sparkContext.parallelize( - """{"struct":{"field1": true, "field2": 92233720368547758070}, """ + - """ "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, """ + - """ "arrayOfString":["str1", "str2"], """ + - """ "arrayOfInteger":[1, 2147483647, -2147483648], """ + - """ "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], """ + - """ "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], """ + - """ "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], """ + - """ "arrayOfBoolean":[true, false, true], """ + - """ "arrayOfNull":[null, null, null, null], """ + - """ "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}], """ + - """ "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], """ + - """ "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] """ + - """ } """ :: Nil) + """{"struct":{"field1": true, "field2": 92233720368547758070}, + "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, + "arrayOfString":["str1", "str2"], + "arrayOfInteger":[1, 2147483647, -2147483648], + "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], + "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], + "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], + "arrayOfBoolean":[true, false, true], + "arrayOfNull":[null, null, null, null], + "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}], + "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], + "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] + }""" :: Nil) val primitiveFieldValueTypeConflict = TestSQLContext.sparkContext.parallelize( - """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, """ + - """ "num_bool":true, "num_str":13.1, "str_bool":"str1"}""" :: - """{"num_num_1":null, "num_num_2":21474836470.9, "num_num_3": null, """ + - """ "num_bool":12, "num_str":null, "str_bool":true}""" :: - """{"num_num_1":21474836470, "num_num_2":92233720368547758070, "num_num_3": 100, """ + - """ "num_bool":false, "num_str":"str1", "str_bool":false}""" :: - """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470, """ + - """ "num_bool":null, "num_str":92233720368547758070, "str_bool":null}""" :: Nil) + """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, + "num_bool":true, "num_str":13.1, "str_bool":"str1"}""" :: + """{"num_num_1":null, "num_num_2":21474836470.9, "num_num_3": null, + "num_bool":12, "num_str":null, "str_bool":true}""" :: + """{"num_num_1":21474836470, "num_num_2":92233720368547758070, "num_num_3": 100, + "num_bool":false, "num_str":"str1", "str_bool":false}""" :: + """{"num_num_1":21474836570, "num_num_2":1.1, "num_num_3": 21474836470, + "num_bool":null, "num_str":92233720368547758070, "str_bool":null}""" :: Nil) val complexFieldValueTypeConflict = TestSQLContext.sparkContext.parallelize( - """{"num_struct":11, "str_array":[1, 2, 3], """ + - """ "array":[], "struct_array":[], "struct": {}}""" :: - """{"num_struct":{"field":false}, "str_array":null, """ + - """ "array":null, "struct_array":{}, "struct": null}""" :: - """{"num_struct":null, "str_array":"str", """ + - """ "array":[4, 5, 6], "struct_array":[7, 8, 9], "struct": {"field":null}}""" :: - """{"num_struct":{}, "str_array":["str1", "str2", 33], """ + - """ "array":[7], "struct_array":{"field": true}, "struct": {"field": "str"}}""" :: Nil) + """{"num_struct":11, "str_array":[1, 2, 3], + "array":[], "struct_array":[], "struct": {}}""" :: + """{"num_struct":{"field":false}, "str_array":null, + "array":null, "struct_array":{}, "struct": null}""" :: + """{"num_struct":null, "str_array":"str", + "array":[4, 5, 6], "struct_array":[7, 8, 9], "struct": {"field":null}}""" :: + """{"num_struct":{}, "str_array":["str1", "str2", 33], + "array":[7], "struct_array":{"field": true}, "struct": {"field": "str"}}""" :: Nil) val arrayElementTypeConflict = TestSQLContext.sparkContext.parallelize( From 8846af532167f54a02639290dc0fc07cde92684e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 10 Jun 2014 10:02:49 -0700 Subject: [PATCH 07/25] API doc. --- .../scala/org/apache/spark/sql/SQLContext.scala | 10 +++++----- .../org/apache/spark/sql/json/JsonTable.scala | 14 ++++++++++++-- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index d5c946c7c2b69..18ca6ff55a9b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -100,8 +100,8 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, parquet.ParquetRelation(path)) /** - * Loads a JSON file, returning the result as a [[SchemaRDD]]. - * Right now, we only do eager schema resolution. + * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. The schema + * of the returned [[SchemaRDD]] is inferred based on the method specified by `mode`. */ def jsonFile( path: String, @@ -112,9 +112,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Loads a RDD[String] storing JSON objects (one object per record), - * returning the result as a [[SchemaRDD]]. - * Right now, we only do eager schema resolution. + * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a + * [[SchemaRDD]]. The schema of the returned [[SchemaRDD]] is inferred based on the method + * specified by `mode`. */ def jsonRDD( json: RDD[String], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 5fb6f0169fb7e..92a82e113fa8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -22,7 +22,7 @@ import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper -import org.apache.spark.annotation.Experimental +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._ @@ -33,8 +33,11 @@ import org.apache.spark.sql.Logging sealed trait SchemaResolutionMode +/** Eagerly infers the schema by scanning the entire dataset.*/ case object EagerSchemaResolution extends SchemaResolutionMode -case class EagerSchemaResolutionWithSampling(fraction: Double) extends SchemaResolutionMode +/** Eagerly infers the schema by scanning a sampled subset of the dataset. The sampling ratio + * is defined by `samplingRatio`.*/ +case class EagerSchemaResolutionWithSampling(samplingRatio: Double) extends SchemaResolutionMode /** * :: Experimental :: @@ -46,6 +49,13 @@ case class EagerSchemaResolutionWithSampling(fraction: Double) extends SchemaRes */ @Experimental object JsonTable extends Logging { + + /** + * Eagerly infers the schema of a given RDD `json` that stores one JSON object per record. + * If a sampled subset of this RDD should be used to infer the schema, `samplingRatio` can + * be used to specify the sampling ratio. + * */ + @DeveloperApi def inferSchema(json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { val schemaData = if (samplingRatio > 0.99) json.sample(false, samplingRatio, 1) else json val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) From 65b87f071e2fb80f03cfea1ab0b9f14342b7e897 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 10 Jun 2014 10:24:32 -0700 Subject: [PATCH 08/25] Fix sampling... --- .../src/main/scala/org/apache/spark/sql/json/JsonTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 92a82e113fa8e..e3c18ae7ede07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -57,7 +57,7 @@ object JsonTable extends Logging { * */ @DeveloperApi def inferSchema(json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { - val schemaData = if (samplingRatio > 0.99) json.sample(false, samplingRatio, 1) else json + val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) // Resolve type conflicts From 43254755b040944ea5380a46825baf008d0e1c09 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 11 Jun 2014 16:15:24 -0700 Subject: [PATCH 09/25] If a sampled dataset is used for schema inferring, update the schema of the JsonTable after first execution. --- examples/src/main/resources/people.json | 3 + .../org/apache/spark/sql/SQLContext.scala | 46 ++-- .../org/apache/spark/sql/SchemaRDD.scala | 12 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 11 +- .../spark/sql/api/java/JavaSchemaRDD.scala | 4 +- .../org/apache/spark/sql/json/JsonTable.scala | 239 ++++++++++++++++-- .../org/apache/spark/sql/json/JsonSuite.scala | 120 ++++++++- .../apache/spark/sql/json/TestJsonData.scala | 2 +- 8 files changed, 366 insertions(+), 71 deletions(-) create mode 100644 examples/src/main/resources/people.json diff --git a/examples/src/main/resources/people.json b/examples/src/main/resources/people.json new file mode 100644 index 0000000000000..6bc4ebf85c775 --- /dev/null +++ b/examples/src/main/resources/people.json @@ -0,0 +1,3 @@ +{"name":Michael} +{"name":Andy, "age":30} +{"name":Justin, "age":19} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 18ca6ff55a9b4..0f620781bcd18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -22,10 +22,8 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration -import org.apache.spark.SparkContext import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD - import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.{ScalaReflection, dsl} import org.apache.spark.sql.catalyst.expressions._ @@ -33,14 +31,12 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.{SetCommand, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor - import org.apache.spark.sql.columnar.InMemoryColumnarTableScan - import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies - -import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.json._ +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.SparkContext /** * :: AlphaComponent :: @@ -100,36 +96,31 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, parquet.ParquetRelation(path)) /** - * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. The schema - * of the returned [[SchemaRDD]] is inferred based on the method specified by `mode`. + * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. + * If a sampled `json` needs to be used, `samplingRatio` can be used to specify + * the sampling ratio. + * + * @group userf */ def jsonFile( path: String, - mode: SchemaResolutionMode = EagerSchemaResolution): SchemaRDD = { - logger.info(s"Loads a JSON file $path.") + samplingRatio: Double = 1.0): JsonTable = { val json = sparkContext.textFile(path) - jsonRDD(json, mode) + jsonRDD(json, samplingRatio) } /** * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a - * [[SchemaRDD]]. The schema of the returned [[SchemaRDD]] is inferred based on the method - * specified by `mode`. + * [[SchemaRDD]]. If a sampled `json` needs to be used, `samplingRatio` can be used to specify + * the sampling ratio. + * + * @group userf */ def jsonRDD( json: RDD[String], - mode: SchemaResolutionMode = EagerSchemaResolution): SchemaRDD = { - mode match { - case EagerSchemaResolution => - logger.info(s"Eagerly resolve the schema without sampling.") - val logicalPlan = JsonTable.inferSchema(json) - logicalPlanToSparkQuery(logicalPlan) - case EagerSchemaResolutionWithSampling(fraction) => - logger.info(s"Eagerly resolve the schema with sampling " + - s"(sampling fraction: $fraction).") - val logicalPlan = JsonTable.inferSchema(json, fraction) - logicalPlanToSparkQuery(logicalPlan) - } + samplingRatio: Double = 1.0): JsonTable = { + val jsonTable = JsonTable(json, samplingRatio, this) + jsonTable } /** @@ -173,6 +164,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { + rdd.tableNames += tableName catalog.registerTable(None, tableName, rdd.logicalPlan) } @@ -192,7 +184,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Returns the specified table as a SchemaRDD */ def table(tableName: String): SchemaRDD = - new SchemaRDD(this, catalog.lookupRelation(None, tableName)) + new SchemaRDD(this, catalog.lookupRelation(None, tableName), Set(tableName)) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { @@ -302,7 +294,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ - protected abstract class QueryExecution { + protected[sql] abstract class QueryExecution { def logical: LogicalPlan def eagerlyProcess(plan: LogicalPlan): RDD[Row] = plan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 8855c4e876917..d7db387e6877a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,6 +19,10 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler +import java.util.{Map => JMap} + +import scala.collection.mutable.HashSet + import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD @@ -30,7 +34,6 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD -import java.util.{Map => JMap} /** * :: AlphaComponent :: @@ -97,11 +100,16 @@ import java.util.{Map => JMap} @AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, - @transient protected[spark] val logicalPlan: LogicalPlan) + @transient protected[spark] val _logicalPlan: LogicalPlan, + @transient private val initialTableNames: Set[String] = Set[String]()) extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike { def baseSchemaRDD = this + val tableNames: HashSet[String] = HashSet[String]() ++ initialTableNames + + protected[spark] def logicalPlan: LogicalPlan = _logicalPlan + // ========================================================================================= // RDD functions: Copy the internal row representation so we present immutable data to users. // ========================================================================================= diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 25e6a10f6e2ae..4c4d71d0d5222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -26,8 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ */ private[sql] trait SchemaRDDLike { @transient val sqlContext: SQLContext - @transient protected[spark] val logicalPlan: LogicalPlan - + protected[spark] def logicalPlan: LogicalPlan private[sql] def baseSchemaRDD: SchemaRDD /** @@ -46,9 +45,11 @@ private[sql] trait SchemaRDDLike { * internally. This object reuse improves performance, but can make programming against the RDD * more difficult. Instead end users should perform RDD operations on a SchemaRDD directly. */ - @transient @DeveloperApi - lazy val queryExecution = sqlContext.executePlan(logicalPlan) + def queryExecution = _queryExecution + + @transient + protected lazy val _queryExecution = sqlContext.executePlan(logicalPlan) override def toString = s"""${super.toString} @@ -115,5 +116,5 @@ private[sql] trait SchemaRDDLike { /** * Print the schema of this SchemaRDD. */ - def printSchema: Unit = queryExecution.analyzed.printSchema() + def printSchema(): Unit = queryExecution.analyzed.printSchema() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index 22f57b758dd02..6035596202b7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -37,12 +37,14 @@ import org.apache.spark.storage.StorageLevel */ class JavaSchemaRDD( @transient val sqlContext: SQLContext, - @transient protected[spark] val logicalPlan: LogicalPlan) + @transient protected[spark] val _logicalPlan: LogicalPlan) extends JavaRDDLike[Row, JavaRDD[Row]] with SchemaRDDLike { private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan) + protected[spark] def logicalPlan: LogicalPlan = _logicalPlan + override val classTag = scala.reflect.classTag[Row] override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index e3c18ae7ede07..141621a08e648 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -18,26 +18,155 @@ package org.apache.spark.sql.json import scala.collection.JavaConversions._ +import scala.collection.mutable.HashSet import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.Accumulable 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 +import org.apache.spark.sql.{Logging, SchemaRDD, SQLContext} -sealed trait SchemaResolutionMode +/** + * A JsonTable is a special kind of [[SchemaRDD]] used to represent a JSON dataset backed by + * a [[RDD]] of strings. Every string is a JSON object. + * + * {{{ + * val sc: SparkContext // An existing spark context. + * val sqlContext = new SQLContext(sc) + * + * // Importing the SQL context gives access to all the SQL functions and implicit conversions. + * import sqlContext._ + * + * // Create a JsonTable from a text file. To infer the schema using the sampled dataset, use + * // jsonFile("examples/src/main/resources/people.json", samplingRatio) + * val jsonTable1 = jsonFile("examples/src/main/resources/people.json") + * rdd.registerAsTable("jsonTable1") + * sql("SELECT * FROM jsonTable2").collect().foreach(println) + * + * // Check if the schema has been updated. Only needed when the initial schema is inferred based + * // on a sampled dataset. + * jsonTable1.adjustSchema() + * + * val rdd = sc.textFile("examples/src/main/resources/people.json") + * // Create a JsonTable from a RDD[String]. To infer the schema using the sampled dataset, use + * // jsonRDD(rdd, samplingRatio) + * val jsonTable2 = jsonRDD(rdd) + * rdd.registerAsTable("jsonTable2") + * sql("SELECT * FROM jsonTable2").collect().foreach(println) + * + * // Check if the schema has been updated. Only needed when the initial schema is inferred based + * // on a sampled dataset. + * jsonTable2.adjustSchema() + * }}} + * + * @groupname json JsonTable Functions + * @groupprio Query -3 + * @groupprio json -2 + * @groupprio schema -1 + */ +protected[sql] class JsonTable( + @transient override val sqlContext: SQLContext, + protected var schema: StructType, + @transient protected var needsAutoCorrection: Boolean, + val jsonRDD: RDD[String]) + extends SchemaRDD(sqlContext, null) { + import org.apache.spark.sql.json.JsonTable._ + + protected var updatedKeysAndTypes = + sqlContext.sparkContext.accumulableCollection(HashSet[(String, DataType)]()) + + protected var errorLogs = sqlContext.sparkContext.accumulableCollection(HashSet[String]()) + + @transient protected var currentLogicalPlan = { + val parsed = if (needsAutoCorrection) { + parseJson(jsonRDD).mapPartitions { + iter => iter.map { record: Map[String, Any] => + updatedKeysAndTypes.localValue ++= getAllKeysWithValueTypes(record) + record + } + } + } else { + parseJson(jsonRDD) + } + + SparkLogicalPlan(ExistingRdd(asAttributes(schema), parsed.map(asRow(_, schema, errorLogs)))) + } + + @transient protected var currentQueryExecution = sqlContext.executePlan(currentLogicalPlan) + + @DeveloperApi + override def queryExecution = { + adjustSchema() + + currentQueryExecution + } + + override protected[spark] def logicalPlan: LogicalPlan = { + adjustSchema() + + currentLogicalPlan + } + + /** + * Checks if we have seen new fields and updated data types during the last execution. If so, + * updates the schema of this JsonTable and returns true. If the schema does not nee + * to be updated, returns false. Also, if there was any runtime exception during last execution + * that has been tolerated, logs the exception at here. If the schema of this JsonTable + * inferred based on a sampled dataset, after the execution of the first query, the user should + * invoke this method to check if the schema has been updated and if the result of the first + * query is not correct because of the partial schema inferred based on sampled dataset. + * + * @group json + */ + def adjustSchema(): Boolean = { + if (!errorLogs.value.isEmpty) { + log.warn("There were runtime errors...") + errorLogs.value.foreach(log.warn(_)) + errorLogs = sqlContext.sparkContext.accumulableCollection(HashSet[String]()) + } + + if (needsAutoCorrection && !updatedKeysAndTypes.value.isEmpty) { + val newSchema = createSchema(updatedKeysAndTypes.value.toSet) + if (schema != newSchema) { + log.info("Schema has been updated.") + println("==== Original Schema ====") + currentLogicalPlan.printSchema() + + // Use the new schema. + schema = newSchema + + // Generate new logical plan. + currentLogicalPlan = + SparkLogicalPlan( + ExistingRdd(asAttributes(schema), parseJson(jsonRDD).map(asRow(_, schema, errorLogs)))) -/** Eagerly infers the schema by scanning the entire dataset.*/ -case object EagerSchemaResolution extends SchemaResolutionMode -/** Eagerly infers the schema by scanning a sampled subset of the dataset. The sampling ratio - * is defined by `samplingRatio`.*/ -case class EagerSchemaResolutionWithSampling(samplingRatio: Double) extends SchemaResolutionMode + println("==== Updated Schema ====") + currentLogicalPlan.printSchema() + + // Create a new accumulable with an empty set. + updatedKeysAndTypes = + sqlContext.sparkContext.accumulableCollection(HashSet[(String, DataType)]()) + currentQueryExecution = sqlContext.executePlan(currentLogicalPlan) + + // Update catalog. + tableNames.foreach { + tableName => sqlContext.catalog.registerTable(None, tableName, currentLogicalPlan) + } + + return true + } + } + + return false + } +} /** * :: Experimental :: @@ -50,16 +179,22 @@ case class EagerSchemaResolutionWithSampling(samplingRatio: Double) extends Sche @Experimental object JsonTable extends Logging { - /** - * Eagerly infers the schema of a given RDD `json` that stores one JSON object per record. - * If a sampled subset of this RDD should be used to infer the schema, `samplingRatio` can - * be used to specify the sampling ratio. - * */ @DeveloperApi - def inferSchema(json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { - val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) + protected[sql] def apply( + json: RDD[String], + samplingRatio: Double = 1.0, + sqlContext: SQLContext): JsonTable = { + require(samplingRatio > 0) + val (schemaData, isSampled) = + if (samplingRatio > 0.99) (json, false) else (json.sample(false, samplingRatio, 1), true) val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) + val schema = createSchema(allKeys) + + new JsonTable(sqlContext, schema, isSampled, json) + } + + protected[json] def createSchema(allKeys: Set[(String, DataType)]): StructType = { // Resolve type conflicts val resolved = allKeys.groupBy { case (key, dataType) => key @@ -119,9 +254,7 @@ object JsonTable extends Logging { StructType(topLevelFields ++ structFields) } - val schema = makeStruct(resolved.keySet.toSeq, Nil) - - SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) + makeStruct(resolved.keySet.toSeq, Nil) } /** @@ -147,6 +280,8 @@ object JsonTable extends Logging { if (fields1 == fields2) => StructType(fields1) 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 } } @@ -204,7 +339,7 @@ object JsonTable extends Logging { * instead of getting all fields of this struct because a field does not appear * in this JSON object can appear in other JSON objects. */ - protected def getAllKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { + protected[json] 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. @@ -251,7 +386,7 @@ object JsonTable extends Logging { case atom => atom } - protected def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { + protected[json] def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { // According to [Jackson-72: https://jira.codehaus.org/browse/JACKSON-72], // ObjectMapper will not return BigDecimal when // "DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS" is disabled @@ -294,6 +429,44 @@ object JsonTable extends Logging { } } + protected def toJsonArrayString(seq: Seq[Any]): String = { + val builder = new StringBuilder + builder.append("[") + var count = 0 + seq.foreach { + element => + if (count > 0) builder.append(",") + count += 1 + builder.append(toString(element)) + } + builder.append("]") + + builder.toString() + } + + protected def toJsonObjectString(map: Map[String, Any]): String = { + val builder = new StringBuilder + builder.append("{") + var count = 0 + map.foreach { + case (key, value) => + if (count > 0) builder.append(",") + count += 1 + builder.append(s"""\"${key}\":${toString(value)}""") + } + builder.append("}") + + builder.toString() + } + + protected def toString(value: Any): String = { + value match { + case value: Map[String, Any] => toJsonObjectString(value) + case value: Seq[Any] => toJsonArrayString(value) + case value => Option(value).map(_.toString).orNull + } + } + protected[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { null @@ -301,7 +474,7 @@ object JsonTable extends Logging { desiredType match { case ArrayType(elementType) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) - case StringType => value.toString + case StringType => toString(value) case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) case DoubleType => toDouble(value) @@ -312,31 +485,45 @@ object JsonTable extends Logging { } } - protected def asRow(json: Map[String,Any], schema: StructType): Row = { + protected[json] def asRow( + json: Map[String,Any], + schema: StructType, + errorLogs: Accumulable[HashSet[String], String]): Row = { val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { // StructType case (StructField(name, fields: StructType, _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( - v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) + v => asRow(v.asInstanceOf[Map[String, Any]], fields, errorLogs)).orNull) // ArrayType(StructType) case (StructField(name, ArrayType(structType: StructType), _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( v => v.asInstanceOf[Seq[Any]].map( - e => asRow(e.asInstanceOf[Map[String, Any]], structType))).orNull) + e => asRow(e.asInstanceOf[Map[String, Any]], structType, errorLogs))).orNull) // Other cases case (StructField(name, dataType, _), i) => - row.update(i, json.get(name).flatMap(v => Option(v)).map( - enforceCorrectType(_, dataType)).getOrElse(null)) + try { + row.update(i, json.get(name).flatMap(v => Option(v)).map( + enforceCorrectType(_, dataType)).getOrElse(null)) + } catch { + case castException: ClassCastException => { + errorLogs.localValue += + s"The original inferred data type (${dataType.toString}) of ${name} is " + + s"not correct. If ${name} is used in the last query, please re-run it to get " + + s"correct results. The exception message is ${castException.toString}" + + row.update(i, null) + } + } } row } - protected def asAttributes(struct: StructType): Seq[AttributeReference] = { + protected[json] def asAttributes(struct: StructType): Seq[AttributeReference] = { struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 2ed95b59e2063..942d0cf1fb4e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,12 +17,17 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.expressions.{Add, Alias, Attribute, AttributeReference} +import scala.collection.mutable.HashSet + +import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.sql.json.JsonTable.enforceCorrectType import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ protected case class Schema(output: Seq[Attribute]) extends LeafNode @@ -310,10 +315,10 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), - (Seq(), "11", "List(1, 2, 3)", "List()", Seq(null)) :: - (null, "Map(field -> false)", null, "Map()", null) :: - (Seq(4, 5, 6), null, "str", "List(7, 8, 9)", Seq(null)) :: - (Seq(7), "Map()","List(str1, str2, 33)", "Map(field -> true)", Seq("str")) :: Nil + (Seq(), "11", "[1,2,3]", "[]", Seq(null)) :: + (null, """{"field":false}""", null, "{}", null) :: + (Seq(4, 5, 6), null, "str", "[7,8,9]", Seq(null)) :: + (Seq(7), "{}","[str1,str2,33]", """{"field":true}""", Seq("str")) :: Nil ) } @@ -329,8 +334,8 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), - Seq(Seq("1", "1.1", "true", null, "List()", "Map()", "List(2, 3, 4)", - "Map(field -> str)")) :: Nil + Seq(Seq("1", "1.1", "true", null, "[]", "{}", "[2,3,4]", + """{"field":str}""")) :: Nil ) // Treat an element as a number. @@ -345,7 +350,7 @@ class JsonSuite extends QueryTest { val expectedSchema = AttributeReference("a", BooleanType, true)() :: - AttributeReference("b", IntegerType, true)() :: + AttributeReference("b", LongType, true)() :: AttributeReference("c", ArrayType(IntegerType), true)() :: AttributeReference("e", StringType, true)() :: AttributeReference("d", StructType( @@ -380,4 +385,101 @@ class JsonSuite extends QueryTest { checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) } + + test("Automatically update the schema when we have missing fields") { + val partialSchema = + StructType(StructField("b", LongType, true) :: StructField("e", StringType, true) :: Nil) + + val json = missingFields + + val jsonTable: JsonTable = + new JsonTable(TestSQLContext, partialSchema, true, json) + + jsonTable.registerAsTable("jsonTable") + + checkAnswer( + sql("select * from jsonTable"), + (null, null) :: + (21474836470L, null) :: + (null, null) :: + (null, null) :: + (null, "str") :: Nil + ) + + // Update the schema + jsonTable.adjustSchema() + + val expectedSchema = + AttributeReference("a", BooleanType, true)() :: + AttributeReference("b", LongType, true)() :: + AttributeReference("c", ArrayType(IntegerType), true)() :: + AttributeReference("e", StringType, true)() :: + AttributeReference("d", StructType( + StructField("field", BooleanType, true) :: Nil), true)() :: Nil + + // We should get the entire schema for this JsonTable. + comparePlans(Schema(expectedSchema), Schema(jsonTable.logicalPlan.output)) + + // The catalog should be updated. + comparePlans(Schema(expectedSchema), + Schema(TestSQLContext.catalog.lookupRelation(None, "jsonTable", None).output)) + + checkAnswer( + jsonTable.select(Star(None)), + (true, null, null, null, null) :: + (null, 21474836470L, null, null, null) :: + (null, null, Seq(33, 44), null, null) :: + (null, null, null, null, Seq(true)) :: + (null, null, null, "str", null) :: Nil + ) + } + + test("Automatically update the schema when we have a wrong type for a primitive field") { + val partialSchema = + StructType(StructField("b", IntegerType, true) :: StructField("e", StringType, true) :: Nil) + + val json = missingFields + + val jsonTable: JsonTable = + new JsonTable(TestSQLContext, partialSchema, true, json) + + jsonTable.registerAsTable("jsonTable") + + // Select all columns of the partial schema. The result is wrong because the type of b is + // IntegerType, but it should be LongType. When we have the wrong type, we return a null. + checkAnswer( + sql("select * from jsonTable"), + (null, null) :: + (null, null) :: + (null, null) :: + (null, null) :: + (null, "str") :: Nil + ) + + // Update the schema + jsonTable.adjustSchema() + + val expectedSchema = + AttributeReference("a", BooleanType, true)() :: + AttributeReference("b", LongType, true)() :: + AttributeReference("c", ArrayType(IntegerType), true)() :: + AttributeReference("e", StringType, true)() :: + AttributeReference("d", StructType( + StructField("field", BooleanType, true) :: Nil), true)() :: Nil + + // We should get the entire schema for this JsonTable. + comparePlans(Schema(expectedSchema), Schema(jsonTable.logicalPlan.output)) + // The catalog should be updated. + comparePlans(Schema(expectedSchema), + Schema(TestSQLContext.catalog.lookupRelation(None, "jsonTable", None).output)) + + checkAnswer( + jsonTable.select(Star(None)), + (true, null, null, null, null) :: + (null, 21474836470L, null, null, null) :: + (null, null, Seq(33, 44), null, null) :: + (null, null, null, null, Seq(true)) :: + (null, null, null, "str", null) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index fe4c95471f8eb..065e04046e8a6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -77,7 +77,7 @@ object TestJsonData { val missingFields = TestSQLContext.sparkContext.parallelize( """{"a":true}""" :: - """{"b":22}""" :: + """{"b":21474836470}""" :: """{"c":[33, 44]}""" :: """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) From 8ffed790e8346b0fa28be261684bb8a532f23702 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 11 Jun 2014 16:37:56 -0700 Subject: [PATCH 10/25] Update the example. --- examples/src/main/resources/people.json | 6 +++--- .../scala/org/apache/spark/sql/json/JsonTable.scala | 10 +++++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/examples/src/main/resources/people.json b/examples/src/main/resources/people.json index 6bc4ebf85c775..50a859cbd7ee8 100644 --- a/examples/src/main/resources/people.json +++ b/examples/src/main/resources/people.json @@ -1,3 +1,3 @@ -{"name":Michael} -{"name":Andy, "age":30} -{"name":Justin, "age":19} +{"name":"Michael"} +{"name":"Andy", "age":30} +{"name":"Justin", "age":19} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 141621a08e648..919e4999da098 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -39,6 +39,8 @@ import org.apache.spark.sql.{Logging, SchemaRDD, SQLContext} * * {{{ * val sc: SparkContext // An existing spark context. + * + * import org.apache.spark.sql.SQLContext * val sqlContext = new SQLContext(sc) * * // Importing the SQL context gives access to all the SQL functions and implicit conversions. @@ -47,8 +49,10 @@ import org.apache.spark.sql.{Logging, SchemaRDD, SQLContext} * // Create a JsonTable from a text file. To infer the schema using the sampled dataset, use * // jsonFile("examples/src/main/resources/people.json", samplingRatio) * val jsonTable1 = jsonFile("examples/src/main/resources/people.json") - * rdd.registerAsTable("jsonTable1") - * sql("SELECT * FROM jsonTable2").collect().foreach(println) + * // Print out the schema + * jsonTable1.printSchema + * jsonTable1.registerAsTable("jsonTable1") + * sql("SELECT * FROM jsonTable1").collect().foreach(println) * * // Check if the schema has been updated. Only needed when the initial schema is inferred based * // on a sampled dataset. @@ -58,7 +62,7 @@ import org.apache.spark.sql.{Logging, SchemaRDD, SQLContext} * // Create a JsonTable from a RDD[String]. To infer the schema using the sampled dataset, use * // jsonRDD(rdd, samplingRatio) * val jsonTable2 = jsonRDD(rdd) - * rdd.registerAsTable("jsonTable2") + * jsonTable2.registerAsTable("jsonTable2") * sql("SELECT * FROM jsonTable2").collect().foreach(println) * * // Check if the schema has been updated. Only needed when the initial schema is inferred based From 66f9e76c0fe27656215ca3377f849977e55844f8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 13 Jun 2014 11:07:23 -0700 Subject: [PATCH 11/25] Update docs and use the entire dataset to infer the schema. --- docs/sql-programming-guide.md | 77 ++++ .../org/apache/spark/sql/SQLContext.scala | 27 +- .../org/apache/spark/sql/SchemaRDD.scala | 12 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 9 +- .../spark/sql/api/java/JavaSchemaRDD.scala | 4 +- .../org/apache/spark/sql/json/JsonTable.scala | 333 ++++++++------- .../org/apache/spark/sql/json/JsonSuite.scala | 399 ++++++++++-------- 7 files changed, 479 insertions(+), 382 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4623bb4247d77..2adb0256b7eb0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -241,6 +241,83 @@ teenNames = teenagers.map(lambda p: "Name: " + p.name) Users that want a more complete dialect of SQL should look at the HiveQL support provided by `HiveContext`. +## Running SQL on JSON Datasets +**Running SQL on JSON datasets is currently only supported in Scala.** + +Spark SQL supports querying JSON datasets. A JSON dataset is represented by [JsonTable](api/scala/index.html#org.apache.spark.sql.json.JsonTable), which is a special kind of SchemaRDD. A JsonTable can be generated with three ways: + +1. A JsonTable can be generated from text files that store one JSON object per line. +2. JsonTable can be generated from a RDD of strings (`RDD[String]`) that stores one JSON object per string. +3. A JsonTable can be generated by unioning two JsonTables. + +The schema (field names and types) of a JsonTable is automatically inferred when this JsonTable is created. + +{% highlight scala %} +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +import sqlContext._ + +// A JSON dataset is pointed by path. +// The path can be either a single text file or a directory storing text files. +val path = "examples/src/main/resources/people.json" +// Create a JsonTable based on the file(s) +val people = jsonFile(path) + +// Because the schema of a JsonTable is automatically inferred, to write queries +// it is better to take a look at what is the schema. +people.printSchema() +// The schema of people is ... +// root +// |-- age: IntegerType +// |-- name: StringType + +// Like a SchemaRDD, a JsonTable can be registered as a table. +people.registerAsTable("people") + +// SQL statements can be run by using the sql methods provided by sqlContext. +val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) + +// Let's create another JsonTable from a RDD[String] storing one JSON object per string. +val anotherPeopleRDD = sc.parallelize( + """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) +val anotherJsonTable = jsonRDD(anotherPeopleRDD) + +// Take a look at the schema of this new JsonTable. +anotherJsonTable.printSchema() +// The schema of anotherJsonTable is ... +// root +// |-- address: StructType +// | |-- city: StringType +// | |-- state: StringType +// |-- name: StringType + +// Two JsonTables can be unioned together. +val unionedJsonTable = people.unionAll(anotherJsonTable) + +// Take a look at the schema of this unioned JsonTable. +unionedJsonTable.printSchema() +// The schema of this unioned JsonTable is ... +// root +// |-- address: StructType +// | |-- city: StringType +// | |-- state: StringType +// |-- age: IntegerType +// |-- name: StringType + +// Register this unioned JsonTable as a table. +unionedJsonTable.registerAsTable("peopleUnioned") +val peopleFromColumbus = + sql("SELECT name, address.city FROM peopleUnioned WHERE address.city = 'Columbus'") + +// See the result. +peopleFromColumbus.map(t => "Name: " + t(0) + ", City: " + t(1)).collect().foreach(println) + +{% endhighlight %} + + ## Using Parquet [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5e355fd35d6ad..1b8019b1ba08a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -102,9 +102,12 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def jsonFile( - path: String, - samplingRatio: Double = 1.0): JsonTable = { + def jsonFile(path: String): JsonTable = jsonFile(path, 1.0) + + /** + * :: Experimental :: + */ + def jsonFile(path: String, samplingRatio: Double): JsonTable = { val json = sparkContext.textFile(path) jsonRDD(json, samplingRatio) } @@ -116,12 +119,13 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def jsonRDD( - json: RDD[String], - samplingRatio: Double = 1.0): JsonTable = { - val jsonTable = JsonTable(json, samplingRatio, this) - jsonTable - } + def jsonRDD(json: RDD[String]): JsonTable = jsonRDD(json, 1.0) + + /** + * :: Experimental :: + */ + def jsonRDD(json: RDD[String], samplingRatio: Double): JsonTable = + JsonTable(this, json, samplingRatio) /** * :: Experimental :: @@ -164,7 +168,6 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - rdd.tableNames += tableName catalog.registerTable(None, tableName, rdd.logicalPlan) } @@ -184,7 +187,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Returns the specified table as a SchemaRDD */ def table(tableName: String): SchemaRDD = - new SchemaRDD(this, catalog.lookupRelation(None, tableName), Set(tableName)) + new SchemaRDD(this, catalog.lookupRelation(None, tableName)) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { @@ -303,7 +306,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ - protected[sql] abstract class QueryExecution { + protected abstract class QueryExecution { def logical: LogicalPlan def eagerlyProcess(plan: LogicalPlan): RDD[Row] = plan match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 1a55fbfcd7fb1..7ad8edf5a5a6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,10 +19,6 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler -import java.util.{Map => JMap} - -import scala.collection.mutable.HashSet - import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD @@ -34,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD +import java.util.{Map => JMap} /** * :: AlphaComponent :: @@ -100,16 +97,11 @@ import org.apache.spark.api.java.JavaRDD @AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, - @transient protected[spark] val _logicalPlan: LogicalPlan, - @transient private val initialTableNames: Set[String] = Set[String]()) + @transient protected[spark] val logicalPlan: LogicalPlan) extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike { def baseSchemaRDD = this - val tableNames: HashSet[String] = HashSet[String]() ++ initialTableNames - - protected[spark] def logicalPlan: LogicalPlan = _logicalPlan - // ========================================================================================= // RDD functions: Copy the internal row representation so we present immutable data to users. // ========================================================================================= diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 4c4d71d0d5222..7f769f7530c75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql.catalyst.plans.logical._ */ private[sql] trait SchemaRDDLike { @transient val sqlContext: SQLContext - protected[spark] def logicalPlan: LogicalPlan + @transient protected[spark] val logicalPlan: LogicalPlan + private[sql] def baseSchemaRDD: SchemaRDD /** @@ -45,11 +46,9 @@ private[sql] trait SchemaRDDLike { * internally. This object reuse improves performance, but can make programming against the RDD * more difficult. Instead end users should perform RDD operations on a SchemaRDD directly. */ - @DeveloperApi - def queryExecution = _queryExecution - @transient - protected lazy val _queryExecution = sqlContext.executePlan(logicalPlan) + @DeveloperApi + lazy val queryExecution = sqlContext.executePlan(logicalPlan) override def toString = s"""${super.toString} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index 6035596202b7d..22f57b758dd02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -37,14 +37,12 @@ import org.apache.spark.storage.StorageLevel */ class JavaSchemaRDD( @transient val sqlContext: SQLContext, - @transient protected[spark] val _logicalPlan: LogicalPlan) + @transient protected[spark] val logicalPlan: LogicalPlan) extends JavaRDDLike[Row, JavaRDD[Row]] with SchemaRDDLike { private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan) - protected[spark] def logicalPlan: LogicalPlan = _logicalPlan - override val classTag = scala.reflect.classTag[Row] override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala index 919e4999da098..603a87ea68469 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala @@ -18,25 +18,27 @@ package org.apache.spark.sql.json import scala.collection.JavaConversions._ -import scala.collection.mutable.HashSet import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper -import org.apache.spark.annotation.{DeveloperApi, Experimental} -import org.apache.spark.Accumulable +import org.apache.spark.annotation.{AlphaComponent, 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, SchemaRDD, SQLContext} +import org.apache.spark.sql.{SQLContext, Logging, SchemaRDD} /** - * A JsonTable is a special kind of [[SchemaRDD]] used to represent a JSON dataset backed by - * a [[RDD]] of strings. Every string is a JSON object. + * :: Experimental :: + * + * A JSON dataset (text files with one JSON object per line or a RDD[String] with one JSON object + * per string) can be directly loaded as a [[SchemaRDD]]. The schema of this [[SchemaRDD]] is + * automatically inferred from the dataset. * + * == SQL Queries == * {{{ * val sc: SparkContext // An existing spark context. * @@ -46,156 +48,142 @@ import org.apache.spark.sql.{Logging, SchemaRDD, SQLContext} * // Importing the SQL context gives access to all the SQL functions and implicit conversions. * import sqlContext._ * - * // Create a JsonTable from a text file. To infer the schema using the sampled dataset, use - * // jsonFile("examples/src/main/resources/people.json", samplingRatio) - * val jsonTable1 = jsonFile("examples/src/main/resources/people.json") - * // Print out the schema - * jsonTable1.printSchema - * jsonTable1.registerAsTable("jsonTable1") - * sql("SELECT * FROM jsonTable1").collect().foreach(println) + * // Create a SchemaRDD from a JSON file (or a directory having JSON files). + * val jsonTable = jsonFile("examples/src/main/resources/people.json") + * // Or, if you have a JSON dataset as RDD[String] + * // val json = sc.textFile("examples/src/main/resources/people.json") + * // val jsonTable = jsonRDD(json) + * + * // See the schema of jsonTable. + * jsonTable.printSchema() + * + * // Register jsonTable as a table. + * jsonTable.registerAsTable("jsonTable") + * + * // Run some queries. + * sql("SELECT name, age FROM jsonTable").collect().foreach(println) + * sql("SELECT name FROM jsonTable WHERE age >= 10 and age <= 19").collect().foreach(println) * - * // Check if the schema has been updated. Only needed when the initial schema is inferred based - * // on a sampled dataset. - * jsonTable1.adjustSchema() + * // Create another RDD[String] storing JSON objects. + * val anotherDataset = sc.parallelize( + * """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) + * val anotherJsonTable = jsonRDD(anotherDataset) * - * val rdd = sc.textFile("examples/src/main/resources/people.json") - * // Create a JsonTable from a RDD[String]. To infer the schema using the sampled dataset, use - * // jsonRDD(rdd, samplingRatio) - * val jsonTable2 = jsonRDD(rdd) - * jsonTable2.registerAsTable("jsonTable2") - * sql("SELECT * FROM jsonTable2").collect().foreach(println) + * // See the schema of anotherJsonTable. + * anotherJsonTable.printSchema() * - * // Check if the schema has been updated. Only needed when the initial schema is inferred based - * // on a sampled dataset. - * jsonTable2.adjustSchema() + * // Union jsonTable and anotherJsonTable. + * val unionedJsonTable = jsonTable.unionAll(anotherJsonTable) + * + * // See the schema of unionedJsonTable. + * unionedJsonTable.printSchema() + * + * // Register unionedJsonTable as table. + * unionedJsonTable.registerAsTable("unionedJsonTable") + * + * // Run some queries. + * sql("SELECT name, age FROM unionedJsonTable").collect().foreach(println) + * sql("SELECT name, age, address FROM unionedJsonTable WHERE address.city = 'Columbus'"). + * collect().foreach(println) * }}} * - * @groupname json JsonTable Functions - * @groupprio Query -3 - * @groupprio json -2 - * @groupprio schema -1 + * == Language Integrated Queries == + * {{{ + * val sc: SparkContext // An existing spark context. + * + * import org.apache.spark.sql.SQLContext + * val sqlContext = new SQLContext(sc) + * + * // Importing the SQL context gives access to all the SQL functions and implicit conversions. + * import sqlContext._ + * + * // Create a SchemaRDD from a JSON file (or a directory having JSON files). + * val jsonTable = jsonFile("examples/src/main/resources/people.json") + * // Or, if you have a JSON dataset as RDD[String] + * // val json = sc.textFile("examples/src/main/resources/people.json") + * // val jsonTable = jsonRDD(json) + * + * // See the schema of jsonTable. + * jsonTable.printSchema() + * + * // Run some queries. + * jsonTable.select('name, 'age).collect().foreach(println) + * jsonTable.where('age <=19).select('name).collect().foreach(println) + * + * // Create another RDD[String] storing JSON objects. + * val anotherDataset = sc.parallelize( + * """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) + * val anotherJsonTable = jsonRDD(anotherDataset) + * + * // See the schema of anotherJsonTable. + * anotherJsonTable.printSchema() + * + * // Union jsonTable and anotherJsonTable. + * val unionedJsonTable = jsonTable.unionAll(anotherJsonTable) + * + * // See the schema of unionedJsonTable. + * unionedJsonTable.printSchema() + * + * // Run some queries. + * unionedJsonTable.select('name, 'age).collect().foreach(println) + * unionedJsonTable.where( + * "address.city".attr === "Columbus").select( + * 'name, 'age, 'address).collect().foreach(println) + * }}} */ -protected[sql] class JsonTable( - @transient override val sqlContext: SQLContext, - protected var schema: StructType, - @transient protected var needsAutoCorrection: Boolean, - val jsonRDD: RDD[String]) - extends SchemaRDD(sqlContext, null) { - import org.apache.spark.sql.json.JsonTable._ - - protected var updatedKeysAndTypes = - sqlContext.sparkContext.accumulableCollection(HashSet[(String, DataType)]()) - - protected var errorLogs = sqlContext.sparkContext.accumulableCollection(HashSet[String]()) - - @transient protected var currentLogicalPlan = { - val parsed = if (needsAutoCorrection) { - parseJson(jsonRDD).mapPartitions { - iter => iter.map { record: Map[String, Any] => - updatedKeysAndTypes.localValue ++= getAllKeysWithValueTypes(record) - record - } - } - } else { - parseJson(jsonRDD) - } - SparkLogicalPlan(ExistingRdd(asAttributes(schema), parsed.map(asRow(_, schema, errorLogs)))) - } - - @transient protected var currentQueryExecution = sqlContext.executePlan(currentLogicalPlan) - - @DeveloperApi - override def queryExecution = { - adjustSchema() - - currentQueryExecution - } - - override protected[spark] def logicalPlan: LogicalPlan = { - adjustSchema() +@AlphaComponent +class JsonTable( + @transient override val sqlContext: SQLContext, + @transient override protected[spark] val logicalPlan: LogicalPlan, + protected[spark] val baseRDD: RDD[String], + protected[spark] val baseSchema: StructType) + extends SchemaRDD(sqlContext, logicalPlan) { - currentLogicalPlan - } + // We widen the type those fields with NullType to StringType. We want to use the original + // schema (baseSchema) with those fields with NullType when we union this JsonTable with another + // JsonTable. + protected[json] lazy val schema = JsonTable.nullTypeToStringType(baseSchema) /** - * Checks if we have seen new fields and updated data types during the last execution. If so, - * updates the schema of this JsonTable and returns true. If the schema does not nee - * to be updated, returns false. Also, if there was any runtime exception during last execution - * that has been tolerated, logs the exception at here. If the schema of this JsonTable - * inferred based on a sampled dataset, after the execution of the first query, the user should - * invoke this method to check if the schema has been updated and if the result of the first - * query is not correct because of the partial schema inferred based on sampled dataset. + * Combines the tuples of two JsonTables and union their schemas, keeping duplicates. * - * @group json + * @group Query */ - def adjustSchema(): Boolean = { - if (!errorLogs.value.isEmpty) { - log.warn("There were runtime errors...") - errorLogs.value.foreach(log.warn(_)) - errorLogs = sqlContext.sparkContext.accumulableCollection(HashSet[String]()) - } - - if (needsAutoCorrection && !updatedKeysAndTypes.value.isEmpty) { - val newSchema = createSchema(updatedKeysAndTypes.value.toSet) - if (schema != newSchema) { - log.info("Schema has been updated.") - println("==== Original Schema ====") - currentLogicalPlan.printSchema() - - // Use the new schema. - schema = newSchema - - // Generate new logical plan. - currentLogicalPlan = - SparkLogicalPlan( - ExistingRdd(asAttributes(schema), parseJson(jsonRDD).map(asRow(_, schema, errorLogs)))) - - println("==== Updated Schema ====") - currentLogicalPlan.printSchema() - - // Create a new accumulable with an empty set. - updatedKeysAndTypes = - sqlContext.sparkContext.accumulableCollection(HashSet[(String, DataType)]()) - currentQueryExecution = sqlContext.executePlan(currentLogicalPlan) - - // Update catalog. - tableNames.foreach { - tableName => sqlContext.catalog.registerTable(None, tableName, currentLogicalPlan) - } - - return true - } - } + def unionAll(otherPlan: JsonTable): JsonTable = { + val unionedBaseSchema = + JsonTable.getCompatibleType(baseSchema, otherPlan.baseSchema).asInstanceOf[StructType] + val unionedJsonRDD = baseRDD.union(otherPlan.baseRDD) + val logicalPlan = JsonTable.createLogicalPlan(unionedJsonRDD, unionedBaseSchema) - return false + new JsonTable(sqlContext, logicalPlan, unionedJsonRDD, unionedBaseSchema) } } -/** - * :: Experimental :: - * Converts a JSON file to a SparkSQL logical query plan. This implementation is only designed to - * work on JSON files that have mostly uniform schema. The conversion suffers from the following - * limitation: - * - The data is optionally sampled to determine all of the possible fields. Any fields that do - * not appear in this sample will not be included in the final output. - */ @Experimental object JsonTable extends Logging { @DeveloperApi protected[sql] def apply( + sqlContext: SQLContext, json: RDD[String], - samplingRatio: Double = 1.0, - sqlContext: SQLContext): JsonTable = { + samplingRatio: Double = 1.0): JsonTable = { require(samplingRatio > 0) - val (schemaData, isSampled) = - if (samplingRatio > 0.99) (json, false) else (json.sample(false, samplingRatio, 1), true) + val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) + val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) - val schema = createSchema(allKeys) + val baseSchema = createSchema(allKeys) - new JsonTable(sqlContext, schema, isSampled, json) + new JsonTable(sqlContext, createLogicalPlan(json, baseSchema), json, baseSchema) + } + + protected[json] def createLogicalPlan( + json: RDD[String], + baseSchema: StructType): LogicalPlan = { + val schema = nullTypeToStringType(baseSchema) + + SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) } protected[json] def createSchema(allKeys: Set[(String, DataType)]): StructType = { @@ -211,14 +199,7 @@ object JsonTable extends Logging { case (_, dataType) => dataType }.reduce((type1: DataType, type2: DataType) => getCompatibleType(type1, type2)) - // Finally, we replace all NullType to StringType. We do not need to take care - // StructType because all fields with a StructType are represented by a placeholder - // StructType(Nil). - dataType match { - case NullType => (fieldName, StringType) - case ArrayType(NullType) => (fieldName, ArrayType(StringType)) - case other => (fieldName, other) - } + (fieldName, dataType) } } @@ -233,8 +214,6 @@ object JsonTable extends Logging { } }.map { a => StructField(a.head, resolved.get(prefix ++ a).get, nullable = true) - }.sortBy { - case StructField(name, _, _) => name } val structFields: Seq[StructField] = structLike.groupBy(_(0)).map { @@ -251,11 +230,12 @@ object JsonTable extends Logging { case StringType => None } } - }.flatMap(field => field).toSeq.sortBy { - case StructField(name, _, _) => name - } + }.flatMap(field => field).toSeq - StructType(topLevelFields ++ structFields) + StructType( + (topLevelFields ++ structFields).sortBy { + case StructField(name, _, _) => name + }) } makeStruct(resolved.keySet.toSeq, Nil) @@ -264,7 +244,7 @@ object JsonTable extends Logging { /** * Returns the most general data type for two given data types. */ - protected def getCompatibleType(t1: DataType, t2: DataType): DataType = { + protected[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)) @@ -279,9 +259,18 @@ object JsonTable extends Logging { (t1, t2) match { case (other: DataType, NullType) => other case (NullType, other: DataType) => other - // TODO: Returns the union of fields1 and fields2? - case (StructType(fields1), StructType(fields2)) - if (fields1 == fields2) => StructType(fields1) + 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 @@ -489,45 +478,55 @@ object JsonTable extends Logging { } } - protected[json] def asRow( - json: Map[String,Any], - schema: StructType, - errorLogs: Accumulable[HashSet[String], String]): Row = { + protected[json] def asRow(json: Map[String,Any], schema: StructType): Row = { val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { // StructType case (StructField(name, fields: StructType, _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( - v => asRow(v.asInstanceOf[Map[String, Any]], fields, errorLogs)).orNull) + v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) // ArrayType(StructType) case (StructField(name, ArrayType(structType: StructType), _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( v => v.asInstanceOf[Seq[Any]].map( - e => asRow(e.asInstanceOf[Map[String, Any]], structType, errorLogs))).orNull) + e => asRow(e.asInstanceOf[Map[String, Any]], structType))).orNull) // Other cases case (StructField(name, dataType, _), i) => - try { - row.update(i, json.get(name).flatMap(v => Option(v)).map( - enforceCorrectType(_, dataType)).getOrElse(null)) - } catch { - case castException: ClassCastException => { - errorLogs.localValue += - s"The original inferred data type (${dataType.toString}) of ${name} is " + - s"not correct. If ${name} is used in the last query, please re-run it to get " + - s"correct results. The exception message is ${castException.toString}" - - row.update(i, null) - } - } + row.update(i, json.get(name).flatMap(v => Option(v)).map( + enforceCorrectType(_, dataType)).getOrElse(null)) } row } + protected def nullTypeToStringType(struct: StructType): StructType = { + val fields = struct.fields.map { + case StructField(fieldName, dataType, nullable) => { + val newType = dataType match { + case NullType => StringType + case ArrayType(NullType) => ArrayType(StringType) + case struct: StructType => nullTypeToStringType(struct) + case other: DataType => other + } + StructField(fieldName, newType, nullable) + } + } + + StructType(fields) + } + protected[json] def asAttributes(struct: StructType): Seq[AttributeReference] = { struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) } + + protected[json] def asStruct(attributes: Seq[AttributeReference]): StructType = { + val fields = attributes.map { + case AttributeReference(name, dataType, nullable) => StructField(name, dataType, nullable) + } + + StructType(fields) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 942d0cf1fb4e9..5816e22f2e787 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,16 +17,11 @@ package org.apache.spark.sql.json -import scala.collection.mutable.HashSet - -import org.apache.spark.sql.catalyst.analysis.Star import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} -import org.apache.spark.sql.json.JsonTable.enforceCorrectType +import org.apache.spark.sql.json.JsonTable.{enforceCorrectType, getCompatibleType} import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ protected case class Schema(output: Seq[Attribute]) extends LeafNode @@ -35,6 +30,136 @@ class JsonSuite extends QueryTest { import TestJsonData._ TestJsonData + test("Type promotion") { + def checkTypePromotion(expected: Any, actual: Any) { + assert(expected.getClass == actual.getClass, + s"Failed to promote ${actual.getClass} to ${expected.getClass}.") + assert(expected == actual, + s"Promoted value ${actual}(${actual.getClass}) does not equal the expected value " + + s"${expected}(${expected.getClass}).") + } + + val intNumber: Int = 2147483647 + checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType)) + checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType)) + checkTypePromotion(intNumber.toDouble, enforceCorrectType(intNumber, DoubleType)) + checkTypePromotion(BigDecimal(intNumber), enforceCorrectType(intNumber, DecimalType)) + + val longNumber: Long = 9223372036854775807L + checkTypePromotion(longNumber, enforceCorrectType(longNumber, LongType)) + checkTypePromotion(longNumber.toDouble, enforceCorrectType(longNumber, DoubleType)) + checkTypePromotion(BigDecimal(longNumber), enforceCorrectType(longNumber, DecimalType)) + + val doubleNumber: Double = 1.7976931348623157E308d + checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) + checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) + } + + test("Get compatible type") { + def checkDataType(t1: DataType, t2: DataType, expected: DataType) { + var actual = getCompatibleType(t1, t2) + assert(actual == expected, + s"Expected $expected as the most general data type for $t1 and $t2, found $actual") + actual = getCompatibleType(t2, t1) + assert(actual == expected, + s"Expected $expected as the most general data type for $t1 and $t2, found $actual") + } + + // NullType + checkDataType(NullType, BooleanType, BooleanType) + checkDataType(NullType, IntegerType, IntegerType) + checkDataType(NullType, LongType, LongType) + checkDataType(NullType, DoubleType, DoubleType) + checkDataType(NullType, DecimalType, DecimalType) + checkDataType(NullType, StringType, StringType) + checkDataType(NullType, ArrayType(IntegerType), ArrayType(IntegerType)) + checkDataType(NullType, StructType(Nil), StructType(Nil)) + checkDataType(NullType, NullType, NullType) + + // BooleanType + checkDataType(BooleanType, BooleanType, BooleanType) + checkDataType(BooleanType, IntegerType, StringType) + checkDataType(BooleanType, LongType, StringType) + checkDataType(BooleanType, DoubleType, StringType) + checkDataType(BooleanType, DecimalType, StringType) + checkDataType(BooleanType, StringType, StringType) + checkDataType(BooleanType, ArrayType(IntegerType), StringType) + checkDataType(BooleanType, StructType(Nil), StringType) + + // IntegerType + checkDataType(IntegerType, IntegerType, IntegerType) + checkDataType(IntegerType, LongType, LongType) + checkDataType(IntegerType, DoubleType, DoubleType) + checkDataType(IntegerType, DecimalType, DecimalType) + checkDataType(IntegerType, StringType, StringType) + checkDataType(IntegerType, ArrayType(IntegerType), StringType) + checkDataType(IntegerType, StructType(Nil), StringType) + + // LongType + checkDataType(LongType, LongType, LongType) + checkDataType(LongType, DoubleType, DoubleType) + checkDataType(LongType, DecimalType, DecimalType) + checkDataType(LongType, StringType, StringType) + checkDataType(LongType, ArrayType(IntegerType), StringType) + checkDataType(LongType, StructType(Nil), StringType) + + // DoubleType + checkDataType(DoubleType, DoubleType, DoubleType) + checkDataType(DoubleType, DecimalType, DecimalType) + checkDataType(DoubleType, StringType, StringType) + checkDataType(DoubleType, ArrayType(IntegerType), StringType) + checkDataType(DoubleType, StructType(Nil), StringType) + + // DoubleType + checkDataType(DecimalType, DecimalType, DecimalType) + checkDataType(DecimalType, StringType, StringType) + checkDataType(DecimalType, ArrayType(IntegerType), StringType) + checkDataType(DecimalType, StructType(Nil), StringType) + + // StringType + checkDataType(StringType, StringType, StringType) + checkDataType(StringType, ArrayType(IntegerType), StringType) + checkDataType(StringType, StructType(Nil), StringType) + + // ArrayType + checkDataType(ArrayType(IntegerType), ArrayType(IntegerType), ArrayType(IntegerType)) + checkDataType(ArrayType(IntegerType), ArrayType(LongType), ArrayType(LongType)) + checkDataType(ArrayType(IntegerType), ArrayType(StringType), ArrayType(StringType)) + checkDataType(ArrayType(IntegerType), StructType(Nil), StringType) + + // StructType + checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) + checkDataType( + StructType(StructField("f1", IntegerType, true) :: Nil), + StructType(StructField("f1", IntegerType, true) :: Nil), + StructType(StructField("f1", IntegerType, true) :: Nil)) + checkDataType( + StructType(StructField("f1", IntegerType, true) :: Nil), + StructType(Nil), + StructType(StructField("f1", IntegerType, true) :: Nil)) + checkDataType( + StructType( + StructField("f1", IntegerType, true) :: + StructField("f2", IntegerType, true) :: Nil), + StructType(StructField("f1", LongType, true) :: Nil) , + StructType( + StructField("f1", LongType, true) :: + StructField("f2", IntegerType, true) :: Nil)) + checkDataType( + StructType( + StructField("f1", IntegerType, true) :: Nil), + StructType( + StructField("f2", IntegerType, true) :: Nil), + StructType( + StructField("f1", IntegerType, true) :: + StructField("f2", IntegerType, true) :: Nil)) + checkDataType( + StructType( + StructField("f1", IntegerType, true) :: Nil), + DecimalType, + StringType) + } + test("Primitive field and type inferring") { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) @@ -132,25 +257,6 @@ class JsonSuite extends QueryTest { (true :: "str1" :: Nil, false :: null :: Nil, null) :: Nil ) - /* - // Right now, "field1" and "field2" are treated as aliases. We should fix it. - // TODO: Re-enable the following test. - checkAnswer( - sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), - (true, "str1") :: Nil - ) - */ - - /* - // Right now, the analyzer cannot resolve arrayOfStruct.field1 and arrayOfStruct.field2. - // TODO: Re-enable the following test. - // Getting all values of a specific field from an array of structs. - checkAnswer( - sql("select arrayOfStruct.field1, arrayOfStruct.field2 from jsonTable"), - (Seq(true, false), Seq("str1", null)) :: Nil - ) - */ - // Access a struct and fields inside of it. checkAnswer( sql("select struct, struct.field1, struct.field2 from jsonTable"), @@ -173,6 +279,24 @@ class JsonSuite extends QueryTest { ) } + ignore("Complex field and type inferring (Ignored)") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType) + jsonSchemaRDD.registerAsTable("jsonTable") + + // Right now, "field1" and "field2" are treated as aliases. We should fix it. + checkAnswer( + sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), + (true, "str1") :: Nil + ) + + // Right now, the analyzer cannot resolve arrayOfStruct.field1 and arrayOfStruct.field2. + // Getting all values of a specific field from an array of structs. + checkAnswer( + sql("select arrayOfStruct.field1, arrayOfStruct.field2 from jsonTable"), + (Seq(true, false), Seq("str1", null)) :: Nil + ) + } + test("Type conflict in primitive field values") { val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) @@ -202,26 +326,6 @@ class JsonSuite extends QueryTest { 2 ) - /* - // Right now, the analyzer does not promote strings in a boolean expreesion. - // Number and Boolean conflict: resolve the type as boolean in this query. - // TODO: Re-enable this test - checkAnswer( - sql("select num_bool from jsonTable where NOT num_bool"), - false - ) - */ - - /* - // Right now, the analyzer does not know that num_bool should be treated as a boolean. - // Number and Boolean conflict: resolve the type as boolean in this query. - // TODO: Re-enable this test - checkAnswer( - sql("select num_bool from jsonTable where num_bool"), - true - ) - */ - // Widening to LongType checkAnswer( sql("select num_num_1 - 100 from jsonTable where num_num_1 > 11"), @@ -245,10 +349,49 @@ class JsonSuite extends QueryTest { Seq(101.2) :: Seq(21474836471.2) :: Nil ) - /* + // Number and String conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 14"), + 92233720368547758071.2 + ) + + // String and Boolean conflict: resolve the type as string. + checkAnswer( + sql("select * from jsonTable where str_bool = 'str1'"), + ("true", 11L, null, 1.1, "13.1", "str1") :: Nil + ) + } + + ignore("Type conflict in primitive field values (Ignored)") { + val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) + jsonSchemaRDD.registerAsTable("jsonTable") + + // Right now, the analyzer does not promote strings in a boolean expreesion. + // Number and Boolean conflict: resolve the type as boolean in this query. + checkAnswer( + sql("select num_bool from jsonTable where NOT num_bool"), + false + ) + + checkAnswer( + sql("select str_bool from jsonTable where NOT str_bool"), + false + ) + + // Right now, the analyzer does not know that num_bool should be treated as a boolean. + // Number and Boolean conflict: resolve the type as boolean in this query. + checkAnswer( + sql("select num_bool from jsonTable where num_bool"), + true + ) + + checkAnswer( + sql("select str_bool from jsonTable where str_bool"), + false + ) + // Right now, we have a parsing error. // Number and String conflict: resolve the type as number in this query. - // TODO: Re-enable this test checkAnswer( sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), BigDecimal("92233720368547758061.2") @@ -266,16 +409,7 @@ class JsonSuite extends QueryTest { select('num_str + 1.2 as Symbol("num")), BigDecimal("92233720368547758061.2") ) - */ - - // Number and String conflict: resolve the type as number in this query. - checkAnswer( - sql("select num_str + 1.2 from jsonTable where num_str > 14"), - 92233720368547758071.2 - // Seq(14.3) :: Seq(92233720368547758071.2) :: Nil - ) - /* // The following test will fail. The type of num_str is StringType. // So, to evaluate num_str + 1.2, we first need to use Cast to convert the type. // In our test data, one value of num_str is 13.1. @@ -286,16 +420,6 @@ class JsonSuite extends QueryTest { sql("select num_str + 1.2 from jsonTable where num_str > 13"), Seq(14.3) :: Seq(92233720368547758071.2) :: Nil ) - */ - - // String and Boolean conflict: resolve the type as string. - checkAnswer( - sql("select * from jsonTable where str_bool = 'str1'"), - ("true", 11L, null, 1.1, "13.1", "str1") :: Nil - ) - - // TODO: Need to test converting str_bool to boolean values. - // Right now, it has the same issues with tests above on num_bool. } test("Type conflict in complex field values") { @@ -305,9 +429,9 @@ class JsonSuite extends QueryTest { AttributeReference("array", ArrayType(IntegerType), true)() :: AttributeReference("num_struct", StringType, true)() :: AttributeReference("str_array", StringType, true)() :: - AttributeReference("struct_array", StringType, true)() :: AttributeReference("struct", StructType( - StructField("field", StringType, true) :: Nil), true)() :: Nil + StructField("field", StringType, true) :: Nil), true)() :: + AttributeReference("struct_array", StringType, true)() :: Nil comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) @@ -315,10 +439,10 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), - (Seq(), "11", "[1,2,3]", "[]", Seq(null)) :: - (null, """{"field":false}""", null, "{}", null) :: - (Seq(4, 5, 6), null, "str", "[7,8,9]", Seq(null)) :: - (Seq(7), "{}","[str1,str2,33]", """{"field":true}""", Seq("str")) :: Nil + (Seq(), "11", "[1,2,3]", Seq(null), "[]") :: + (null, """{"field":false}""", null, null, "{}") :: + (Seq(4, 5, 6), null, "str", Seq(null), "[7,8,9]") :: + (Seq(7), "{}","[str1,str2,33]", Seq("str"), """{"field":true}""") :: Nil ) } @@ -352,134 +476,39 @@ class JsonSuite extends QueryTest { AttributeReference("a", BooleanType, true)() :: AttributeReference("b", LongType, true)() :: AttributeReference("c", ArrayType(IntegerType), true)() :: - AttributeReference("e", StringType, true)() :: AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: Nil + StructField("field", BooleanType, true) :: Nil), true)() :: + AttributeReference("e", StringType, true)() :: Nil comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) jsonSchemaRDD.registerAsTable("jsonTable") } - test("Type promotion") { - def checkTypePromotion(expected: Any, actual: Any) { - assert(expected.getClass == actual.getClass, - s"Failed to promote ${actual.getClass} to ${expected.getClass}.") - assert(expected == actual, - s"Promoted value ${actual}(${actual.getClass}) does not equal the expected value " + - s"${expected}(${expected.getClass}).") - } - - val intNumber: Int = 2147483647 - checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType)) - checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType)) - checkTypePromotion(intNumber.toDouble, enforceCorrectType(intNumber, DoubleType)) - checkTypePromotion(BigDecimal(intNumber), enforceCorrectType(intNumber, DecimalType)) - - val longNumber: Long = 9223372036854775807L - checkTypePromotion(longNumber, enforceCorrectType(longNumber, LongType)) - checkTypePromotion(longNumber.toDouble, enforceCorrectType(longNumber, DoubleType)) - checkTypePromotion(BigDecimal(longNumber), enforceCorrectType(longNumber, DecimalType)) - - val doubleNumber: Double = 1.7976931348623157E308d - checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) - checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) - } - - test("Automatically update the schema when we have missing fields") { - val partialSchema = - StructType(StructField("b", LongType, true) :: StructField("e", StringType, true) :: Nil) - - val json = missingFields - - val jsonTable: JsonTable = - new JsonTable(TestSQLContext, partialSchema, true, json) - - jsonTable.registerAsTable("jsonTable") - - checkAnswer( - sql("select * from jsonTable"), - (null, null) :: - (21474836470L, null) :: - (null, null) :: - (null, null) :: - (null, "str") :: Nil - ) - - // Update the schema - jsonTable.adjustSchema() + test("Union JsonTables") { + val jsonSchemaRDD1 = jsonRDD(primitiveFieldAndType) + val jsonSchemaRDD2 = jsonRDD(missingFields) val expectedSchema = AttributeReference("a", BooleanType, true)() :: AttributeReference("b", LongType, true)() :: + AttributeReference("bigInteger", DecimalType, true)() :: + AttributeReference("boolean", BooleanType, true)() :: AttributeReference("c", ArrayType(IntegerType), true)() :: - AttributeReference("e", StringType, true)() :: AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: Nil - - // We should get the entire schema for this JsonTable. - comparePlans(Schema(expectedSchema), Schema(jsonTable.logicalPlan.output)) - - // The catalog should be updated. - comparePlans(Schema(expectedSchema), - Schema(TestSQLContext.catalog.lookupRelation(None, "jsonTable", None).output)) - - checkAnswer( - jsonTable.select(Star(None)), - (true, null, null, null, null) :: - (null, 21474836470L, null, null, null) :: - (null, null, Seq(33, 44), null, null) :: - (null, null, null, null, Seq(true)) :: - (null, null, null, "str", null) :: Nil - ) - } - - test("Automatically update the schema when we have a wrong type for a primitive field") { - val partialSchema = - StructType(StructField("b", IntegerType, true) :: StructField("e", StringType, true) :: Nil) - - val json = missingFields - - val jsonTable: JsonTable = - new JsonTable(TestSQLContext, partialSchema, true, json) - - jsonTable.registerAsTable("jsonTable") - - // Select all columns of the partial schema. The result is wrong because the type of b is - // IntegerType, but it should be LongType. When we have the wrong type, we return a null. - checkAnswer( - sql("select * from jsonTable"), - (null, null) :: - (null, null) :: - (null, null) :: - (null, null) :: - (null, "str") :: Nil - ) - - // Update the schema - jsonTable.adjustSchema() - - val expectedSchema = - AttributeReference("a", BooleanType, true)() :: - AttributeReference("b", LongType, true)() :: - AttributeReference("c", ArrayType(IntegerType), true)() :: + StructField("field", BooleanType, true) :: Nil), true)() :: + AttributeReference("double", DoubleType, true)() :: AttributeReference("e", StringType, true)() :: - AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: Nil + AttributeReference("integer", IntegerType, true)() :: + AttributeReference("long", LongType, true)() :: + AttributeReference("null", StringType, true)() :: + AttributeReference("string", StringType, true)() :: Nil - // We should get the entire schema for this JsonTable. - comparePlans(Schema(expectedSchema), Schema(jsonTable.logicalPlan.output)) - // The catalog should be updated. - comparePlans(Schema(expectedSchema), - Schema(TestSQLContext.catalog.lookupRelation(None, "jsonTable", None).output)) + val unioned1 = jsonSchemaRDD1.unionAll(jsonSchemaRDD2) + comparePlans(Schema(expectedSchema), Schema(unioned1.logicalPlan.output)) - checkAnswer( - jsonTable.select(Star(None)), - (true, null, null, null, null) :: - (null, 21474836470L, null, null, null) :: - (null, null, Seq(33, 44), null, null) :: - (null, null, null, null, Seq(true)) :: - (null, null, null, "str", null) :: Nil - ) + val unioned2 = jsonSchemaRDD1.unionAll(jsonSchemaRDD2) + comparePlans(Schema(expectedSchema), Schema(unioned2.logicalPlan.output)) } + } From 6df0891bb7939173403d295bd00fe911f7c4917e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 13 Jun 2014 11:48:16 -0700 Subject: [PATCH 12/25] Apache header. --- .../spark/sql/catalyst/plans/PlanTest.scala | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index bc808a138efb8..7e9f47ef21df8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -1,3 +1,20 @@ +/* + * 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.plans import org.scalatest.FunSuite From ab810b0968f53f9869e0d30b659758bf453023b7 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 13 Jun 2014 14:25:23 -0700 Subject: [PATCH 13/25] Make JsonRDD private. --- docs/sql-programming-guide.md | 40 +++++----- .../org/apache/spark/sql/SQLContext.scala | 10 +-- .../org/apache/spark/sql/SchemaRDD.scala | 6 +- .../json/{JsonTable.scala => JsonRDD.scala} | 73 ++++++++++--------- .../org/apache/spark/sql/json/JsonSuite.scala | 2 +- 5 files changed, 69 insertions(+), 62 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/json/{JsonTable.scala => JsonRDD.scala} (89%) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2adb0256b7eb0..2b0c4c5414ff2 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -244,13 +244,13 @@ Users that want a more complete dialect of SQL should look at the HiveQL support ## Running SQL on JSON Datasets **Running SQL on JSON datasets is currently only supported in Scala.** -Spark SQL supports querying JSON datasets. A JSON dataset is represented by [JsonTable](api/scala/index.html#org.apache.spark.sql.json.JsonTable), which is a special kind of SchemaRDD. A JsonTable can be generated with three ways: +Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are three ways: -1. A JsonTable can be generated from text files that store one JSON object per line. -2. JsonTable can be generated from a RDD of strings (`RDD[String]`) that stores one JSON object per string. -3. A JsonTable can be generated by unioning two JsonTables. +1. Creating the SchemaRDD from text files that store one JSON object per line by using `jsonFile`. +2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object per string by using `jsonRDD`. +3. Creating the SchemaRDD by unioning two SchemaRDDs for two JSON datasets through `schemaRDD1.unionAll(schemaRDD2)`. -The schema (field names and types) of a JsonTable is automatically inferred when this JsonTable is created. +The schema (field names and types) of a JSON dataset is automatically inferred when the SchemaRDD is created. {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -259,10 +259,10 @@ import sqlContext._ // A JSON dataset is pointed by path. // The path can be either a single text file or a directory storing text files. val path = "examples/src/main/resources/people.json" -// Create a JsonTable based on the file(s) +// Create a SchemaRDD from the file(s) pointed by path val people = jsonFile(path) -// Because the schema of a JsonTable is automatically inferred, to write queries +// Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. people.printSchema() // The schema of people is ... @@ -270,7 +270,7 @@ people.printSchema() // |-- age: IntegerType // |-- name: StringType -// Like a SchemaRDD, a JsonTable can be registered as a table. +// Register this SchemaRDD as a table. people.registerAsTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. @@ -280,26 +280,26 @@ val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // The columns of a row in the result can be accessed by ordinal. teenagers.map(t => "Name: " + t(0)).collect().foreach(println) -// Let's create another JsonTable from a RDD[String] storing one JSON object per string. +// Let's create another SchemaRDD from a RDD[String] storing one JSON object per string. val anotherPeopleRDD = sc.parallelize( """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) -val anotherJsonTable = jsonRDD(anotherPeopleRDD) +val anotherPeople = jsonRDD(anotherPeopleRDD) -// Take a look at the schema of this new JsonTable. -anotherJsonTable.printSchema() -// The schema of anotherJsonTable is ... +// Take a look at the schema of this new SchemaRDD. +anotherPeople.printSchema() +// The schema of anotherPeople is ... // root // |-- address: StructType // | |-- city: StringType // | |-- state: StringType // |-- name: StringType -// Two JsonTables can be unioned together. -val unionedJsonTable = people.unionAll(anotherJsonTable) +// Two SchemaRDDs created from JSON datasets can be unioned together. +val peopleUnioned = people.unionAll(anotherPeople) -// Take a look at the schema of this unioned JsonTable. -unionedJsonTable.printSchema() -// The schema of this unioned JsonTable is ... +// Take a look at the schema of this unioned SchemaRDD. +peopleUnioned.printSchema() +// The schema of this unioned SchemaRDD is ... // root // |-- address: StructType // | |-- city: StringType @@ -307,8 +307,8 @@ unionedJsonTable.printSchema() // |-- age: IntegerType // |-- name: StringType -// Register this unioned JsonTable as a table. -unionedJsonTable.registerAsTable("peopleUnioned") +// Register this unioned SchemaRDD as a table. +peopleUnioned.registerAsTable("peopleUnioned") val peopleFromColumbus = sql("SELECT name, address.city FROM peopleUnioned WHERE address.city = 'Columbus'") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index f86efc05aac91..6b00d834a1ec3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -102,12 +102,12 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def jsonFile(path: String): JsonTable = jsonFile(path, 1.0) + def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0) /** * :: Experimental :: */ - def jsonFile(path: String, samplingRatio: Double): JsonTable = { + def jsonFile(path: String, samplingRatio: Double): SchemaRDD = { val json = sparkContext.textFile(path) jsonRDD(json, samplingRatio) } @@ -119,13 +119,13 @@ class SQLContext(@transient val sparkContext: SparkContext) * * @group userf */ - def jsonRDD(json: RDD[String]): JsonTable = jsonRDD(json, 1.0) + def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0) /** * :: Experimental :: */ - def jsonRDD(json: RDD[String], samplingRatio: Double): JsonTable = - JsonTable(this, json, samplingRatio) + def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = + JsonRDD(this, json, samplingRatio) /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 7ad8edf5a5a6e..db3e7ab2cd7e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -41,8 +41,10 @@ import java.util.{Map => JMap} * whose elements are scala case classes into a SchemaRDD. This conversion can also be done * explicitly using the `createSchemaRDD` function on a [[SQLContext]]. * - * A `SchemaRDD` can also be created by loading data in from external sources, for example, - * by using the `parquetFile` method on [[SQLContext]]. + * A `SchemaRDD` can also be created by loading data in from external sources. + * Examples are loading data from Parquet files by using by using the + * `parquetFile` method on [[SQLContext]], and loading JSON datasets + * by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. * * == SQL Queries == * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala similarity index 89% rename from sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala rename to sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 603a87ea68469..387f31b4fcd8f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -22,7 +22,7 @@ import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper -import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} +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._ @@ -132,8 +132,8 @@ import org.apache.spark.sql.{SQLContext, Logging, SchemaRDD} * }}} */ -@AlphaComponent -class JsonTable( +@Experimental +private class JsonRDD( @transient override val sqlContext: SQLContext, @transient override protected[spark] val logicalPlan: LogicalPlan, protected[spark] val baseRDD: RDD[String], @@ -143,31 +143,36 @@ class JsonTable( // We widen the type those fields with NullType to StringType. We want to use the original // schema (baseSchema) with those fields with NullType when we union this JsonTable with another // JsonTable. - protected[json] lazy val schema = JsonTable.nullTypeToStringType(baseSchema) + lazy val schema = JsonRDD.nullTypeToStringType(baseSchema) /** * Combines the tuples of two JsonTables and union their schemas, keeping duplicates. * * @group Query */ - def unionAll(otherPlan: JsonTable): JsonTable = { - val unionedBaseSchema = - JsonTable.getCompatibleType(baseSchema, otherPlan.baseSchema).asInstanceOf[StructType] - val unionedJsonRDD = baseRDD.union(otherPlan.baseRDD) - val logicalPlan = JsonTable.createLogicalPlan(unionedJsonRDD, unionedBaseSchema) - - new JsonTable(sqlContext, logicalPlan, unionedJsonRDD, unionedBaseSchema) + override def unionAll(otherPlan: SchemaRDD): SchemaRDD = { + otherPlan match { + case jsonRDD: JsonRDD => { + val unionedBaseSchema = + JsonRDD.getCompatibleType(baseSchema, jsonRDD.baseSchema).asInstanceOf[StructType] + val unionedJsonRDD = baseRDD.union(jsonRDD.baseRDD) + val logicalPlan = JsonRDD.createLogicalPlan(unionedJsonRDD, unionedBaseSchema) + + new JsonRDD(sqlContext, logicalPlan, unionedJsonRDD, unionedBaseSchema) + } + case schemaRDD: SchemaRDD => super.unionAll(otherPlan) + } } } @Experimental -object JsonTable extends Logging { +private[sql] object JsonRDD extends Logging { @DeveloperApi - protected[sql] def apply( + private[sql] def apply( sqlContext: SQLContext, json: RDD[String], - samplingRatio: Double = 1.0): JsonTable = { + samplingRatio: Double = 1.0): SchemaRDD = { require(samplingRatio > 0) val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) @@ -175,10 +180,10 @@ object JsonTable extends Logging { val baseSchema = createSchema(allKeys) - new JsonTable(sqlContext, createLogicalPlan(json, baseSchema), json, baseSchema) + new JsonRDD(sqlContext, createLogicalPlan(json, baseSchema), json, baseSchema) } - protected[json] def createLogicalPlan( + private def createLogicalPlan( json: RDD[String], baseSchema: StructType): LogicalPlan = { val schema = nullTypeToStringType(baseSchema) @@ -186,7 +191,7 @@ object JsonTable extends Logging { SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) } - protected[json] def createSchema(allKeys: Set[(String, DataType)]): StructType = { + private def createSchema(allKeys: Set[(String, DataType)]): StructType = { // Resolve type conflicts val resolved = allKeys.groupBy { case (key, dataType) => key @@ -244,7 +249,7 @@ object JsonTable extends Logging { /** * Returns the most general data type for two given data types. */ - protected[json] def getCompatibleType(t1: DataType, t2: DataType): DataType = { + 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)) @@ -280,7 +285,7 @@ object JsonTable extends Logging { } } - protected def getPrimitiveType(value: Any): DataType = { + private def getPrimitiveType(value: Any): DataType = { value match { case value: java.lang.String => StringType case value: java.lang.Integer => IntegerType @@ -303,7 +308,7 @@ object JsonTable extends Logging { * type conflicts. Right now, when the element of an array is another array, we * treat the element as String. */ - protected def getTypeOfArray(l: Seq[Any]): ArrayType = { + 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. @@ -332,7 +337,7 @@ object JsonTable extends Logging { * instead of getting all fields of this struct because a field does not appear * in this JSON object can appear in other JSON objects. */ - protected[json] def getAllKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { + 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. @@ -368,7 +373,7 @@ object JsonTable extends Logging { * DefaultScalaModule in jackson-module-scala will make * the parsing very slow. */ - protected def scalafy(obj: Any): Any = obj match { + private def scalafy(obj: Any): Any = obj match { case map: java.util.Map[String, Object] => // .map(identity) is used as a workaround of non-serializable Map // generated by .mapValues. @@ -379,7 +384,7 @@ object JsonTable extends Logging { case atom => atom } - protected[json] def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { + private def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { // According to [Jackson-72: https://jira.codehaus.org/browse/JACKSON-72], // ObjectMapper will not return BigDecimal when // "DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS" is disabled @@ -397,14 +402,14 @@ object JsonTable extends Logging { }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) } - protected def toLong(value: Any): Long = { + private def toLong(value: Any): Long = { value match { case value: java.lang.Integer => value.asInstanceOf[Int].toLong case value: java.lang.Long => value.asInstanceOf[Long] } } - protected def toDouble(value: Any): Double = { + private def toDouble(value: Any): Double = { value match { case value: java.lang.Integer => value.asInstanceOf[Int].toDouble case value: java.lang.Long => value.asInstanceOf[Long].toDouble @@ -412,7 +417,7 @@ object JsonTable extends Logging { } } - protected def toDecimal(value: Any): BigDecimal = { + private def toDecimal(value: Any): BigDecimal = { value match { case value: java.lang.Integer => BigDecimal(value) case value: java.lang.Long => BigDecimal(value) @@ -422,7 +427,7 @@ object JsonTable extends Logging { } } - protected def toJsonArrayString(seq: Seq[Any]): String = { + private def toJsonArrayString(seq: Seq[Any]): String = { val builder = new StringBuilder builder.append("[") var count = 0 @@ -437,7 +442,7 @@ object JsonTable extends Logging { builder.toString() } - protected def toJsonObjectString(map: Map[String, Any]): String = { + private def toJsonObjectString(map: Map[String, Any]): String = { val builder = new StringBuilder builder.append("{") var count = 0 @@ -452,7 +457,7 @@ object JsonTable extends Logging { builder.toString() } - protected def toString(value: Any): String = { + private def toString(value: Any): String = { value match { case value: Map[String, Any] => toJsonObjectString(value) case value: Seq[Any] => toJsonArrayString(value) @@ -460,7 +465,7 @@ object JsonTable extends Logging { } } - protected[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ + private[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { null } else { @@ -478,7 +483,7 @@ object JsonTable extends Logging { } } - protected[json] def asRow(json: Map[String,Any], schema: StructType): Row = { + private def asRow(json: Map[String,Any], schema: StructType): Row = { val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { // StructType @@ -502,7 +507,7 @@ object JsonTable extends Logging { row } - protected def nullTypeToStringType(struct: StructType): StructType = { + private def nullTypeToStringType(struct: StructType): StructType = { val fields = struct.fields.map { case StructField(fieldName, dataType, nullable) => { val newType = dataType match { @@ -518,11 +523,11 @@ object JsonTable extends Logging { StructType(fields) } - protected[json] def asAttributes(struct: StructType): Seq[AttributeReference] = { + private def asAttributes(struct: StructType): Seq[AttributeReference] = { struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) } - protected[json] def asStruct(attributes: Seq[AttributeReference]): StructType = { + private def asStruct(attributes: Seq[AttributeReference]): StructType = { val fields = attributes.map { case AttributeReference(name, dataType, nullable) => StructField(name, dataType, nullable) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 5816e22f2e787..a148a641597f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.json import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.json.JsonTable.{enforceCorrectType, getCompatibleType} +import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, getCompatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ From cff84ccce419a5208ac06392f20b874640921fd9 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 09:59:53 -0700 Subject: [PATCH 14/25] Use a SchemaRDD for a JSON dataset. --- .../spark/sql/catalyst/plans/QueryPlan.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 12 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 9 +- .../org/apache/spark/sql/json/JsonRDD.scala | 143 +----------------- .../org/apache/spark/sql/json/JsonSuite.scala | 47 +++--- 5 files changed, 41 insertions(+), 172 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index a7cb4ec87784e..9b34f6e0bbf31 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -129,5 +129,5 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy def getSchemaTreeString(): String = plans.generateSchemaTreeString(output) /** Prints out the schema in the tree format */ - def printSchema(): Unit = println(getSchemaTreeString) + def printSchema(): Unit = println(getSchemaTreeString()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 559bc24bc2c84..1b71766e28ffd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -25,8 +25,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.{ScalaReflection, dsl} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.dsl.ExpressionConversions import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -50,7 +51,7 @@ import org.apache.spark.SparkContext class SQLContext(@transient val sparkContext: SparkContext) extends Logging with SQLConf - with dsl.ExpressionConversions + with ExpressionConversions with Serializable { self => @@ -97,8 +98,6 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. - * If a sampled `json` needs to be used, `samplingRatio` can be used to specify - * the sampling ratio. * * @group userf */ @@ -114,8 +113,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a - * [[SchemaRDD]]. If a sampled `json` needs to be used, `samplingRatio` can be used to specify - * the sampling ratio. + * [[SchemaRDD]]. * * @group userf */ @@ -125,7 +123,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * :: Experimental :: */ def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = - JsonRDD(this, json, samplingRatio) + new SchemaRDD(this, JsonRDD.inferSchema(json, samplingRatio)) /** * :: Experimental :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index ba3aef660edaf..206063ff53c35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -123,8 +123,9 @@ private[sql] trait SchemaRDDLike { def saveAsTable(tableName: String): Unit = sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd - /** - * Print the schema of this SchemaRDD. - */ - def printSchema(): Unit = queryExecution.analyzed.printSchema() + /** Returns the output schema in the tree format. */ + def getSchemaTreeString(): String = queryExecution.analyzed.getSchemaTreeString() + + /** Prints out the schema in the tree format. */ + def printSchema(): Unit = println(getSchemaTreeString()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 387f31b4fcd8f..ca912d33e4548 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -29,150 +29,15 @@ 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.{SQLContext, Logging, SchemaRDD} - -/** - * :: Experimental :: - * - * A JSON dataset (text files with one JSON object per line or a RDD[String] with one JSON object - * per string) can be directly loaded as a [[SchemaRDD]]. The schema of this [[SchemaRDD]] is - * automatically inferred from the dataset. - * - * == SQL Queries == - * {{{ - * val sc: SparkContext // An existing spark context. - * - * import org.apache.spark.sql.SQLContext - * val sqlContext = new SQLContext(sc) - * - * // Importing the SQL context gives access to all the SQL functions and implicit conversions. - * import sqlContext._ - * - * // Create a SchemaRDD from a JSON file (or a directory having JSON files). - * val jsonTable = jsonFile("examples/src/main/resources/people.json") - * // Or, if you have a JSON dataset as RDD[String] - * // val json = sc.textFile("examples/src/main/resources/people.json") - * // val jsonTable = jsonRDD(json) - * - * // See the schema of jsonTable. - * jsonTable.printSchema() - * - * // Register jsonTable as a table. - * jsonTable.registerAsTable("jsonTable") - * - * // Run some queries. - * sql("SELECT name, age FROM jsonTable").collect().foreach(println) - * sql("SELECT name FROM jsonTable WHERE age >= 10 and age <= 19").collect().foreach(println) - * - * // Create another RDD[String] storing JSON objects. - * val anotherDataset = sc.parallelize( - * """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) - * val anotherJsonTable = jsonRDD(anotherDataset) - * - * // See the schema of anotherJsonTable. - * anotherJsonTable.printSchema() - * - * // Union jsonTable and anotherJsonTable. - * val unionedJsonTable = jsonTable.unionAll(anotherJsonTable) - * - * // See the schema of unionedJsonTable. - * unionedJsonTable.printSchema() - * - * // Register unionedJsonTable as table. - * unionedJsonTable.registerAsTable("unionedJsonTable") - * - * // Run some queries. - * sql("SELECT name, age FROM unionedJsonTable").collect().foreach(println) - * sql("SELECT name, age, address FROM unionedJsonTable WHERE address.city = 'Columbus'"). - * collect().foreach(println) - * }}} - * - * == Language Integrated Queries == - * {{{ - * val sc: SparkContext // An existing spark context. - * - * import org.apache.spark.sql.SQLContext - * val sqlContext = new SQLContext(sc) - * - * // Importing the SQL context gives access to all the SQL functions and implicit conversions. - * import sqlContext._ - * - * // Create a SchemaRDD from a JSON file (or a directory having JSON files). - * val jsonTable = jsonFile("examples/src/main/resources/people.json") - * // Or, if you have a JSON dataset as RDD[String] - * // val json = sc.textFile("examples/src/main/resources/people.json") - * // val jsonTable = jsonRDD(json) - * - * // See the schema of jsonTable. - * jsonTable.printSchema() - * - * // Run some queries. - * jsonTable.select('name, 'age).collect().foreach(println) - * jsonTable.where('age <=19).select('name).collect().foreach(println) - * - * // Create another RDD[String] storing JSON objects. - * val anotherDataset = sc.parallelize( - * """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) - * val anotherJsonTable = jsonRDD(anotherDataset) - * - * // See the schema of anotherJsonTable. - * anotherJsonTable.printSchema() - * - * // Union jsonTable and anotherJsonTable. - * val unionedJsonTable = jsonTable.unionAll(anotherJsonTable) - * - * // See the schema of unionedJsonTable. - * unionedJsonTable.printSchema() - * - * // Run some queries. - * unionedJsonTable.select('name, 'age).collect().foreach(println) - * unionedJsonTable.where( - * "address.city".attr === "Columbus").select( - * 'name, 'age, 'address).collect().foreach(println) - * }}} - */ - -@Experimental -private class JsonRDD( - @transient override val sqlContext: SQLContext, - @transient override protected[spark] val logicalPlan: LogicalPlan, - protected[spark] val baseRDD: RDD[String], - protected[spark] val baseSchema: StructType) - extends SchemaRDD(sqlContext, logicalPlan) { - - // We widen the type those fields with NullType to StringType. We want to use the original - // schema (baseSchema) with those fields with NullType when we union this JsonTable with another - // JsonTable. - lazy val schema = JsonRDD.nullTypeToStringType(baseSchema) - - /** - * Combines the tuples of two JsonTables and union their schemas, keeping duplicates. - * - * @group Query - */ - override def unionAll(otherPlan: SchemaRDD): SchemaRDD = { - otherPlan match { - case jsonRDD: JsonRDD => { - val unionedBaseSchema = - JsonRDD.getCompatibleType(baseSchema, jsonRDD.baseSchema).asInstanceOf[StructType] - val unionedJsonRDD = baseRDD.union(jsonRDD.baseRDD) - val logicalPlan = JsonRDD.createLogicalPlan(unionedJsonRDD, unionedBaseSchema) - - new JsonRDD(sqlContext, logicalPlan, unionedJsonRDD, unionedBaseSchema) - } - case schemaRDD: SchemaRDD => super.unionAll(otherPlan) - } - } -} +import org.apache.spark.sql.Logging @Experimental private[sql] object JsonRDD extends Logging { @DeveloperApi - private[sql] def apply( - sqlContext: SQLContext, + private[sql] def inferSchema( json: RDD[String], - samplingRatio: Double = 1.0): SchemaRDD = { + samplingRatio: Double = 1.0): LogicalPlan = { require(samplingRatio > 0) val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) @@ -180,7 +45,7 @@ private[sql] object JsonRDD extends Logging { val baseSchema = createSchema(allKeys) - new JsonRDD(sqlContext, createLogicalPlan(json, baseSchema), json, baseSchema) + createLogicalPlan(json, baseSchema) } private def createLogicalPlan( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index a148a641597f5..79a9ba8fe7dcf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.json import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, getCompatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ @@ -469,7 +470,7 @@ class JsonSuite extends QueryTest { ) } - test("Handle missing fields") { + test("Handling missing fields") { val jsonSchemaRDD = jsonRDD(missingFields) val expectedSchema = @@ -485,30 +486,34 @@ class JsonSuite extends QueryTest { jsonSchemaRDD.registerAsTable("jsonTable") } - test("Union JsonTables") { - val jsonSchemaRDD1 = jsonRDD(primitiveFieldAndType) - val jsonSchemaRDD2 = jsonRDD(missingFields) + test("Loading a JSON dataset from a text file") { + val file = getTempFilePath("json") + val path = file.toString + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + val jsonSchemaRDD = jsonFile(path) val expectedSchema = - AttributeReference("a", BooleanType, true)() :: - AttributeReference("b", LongType, true)() :: AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("c", ArrayType(IntegerType), true)() :: - AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("e", StringType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + AttributeReference("boolean", BooleanType, true)() :: + AttributeReference("double", DoubleType, true)() :: + AttributeReference("integer", IntegerType, true)() :: + AttributeReference("long", LongType, true)() :: + AttributeReference("null", StringType, true)() :: + AttributeReference("string", StringType, true)() :: Nil - val unioned1 = jsonSchemaRDD1.unionAll(jsonSchemaRDD2) - comparePlans(Schema(expectedSchema), Schema(unioned1.logicalPlan.output)) + comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) - val unioned2 = jsonSchemaRDD1.unionAll(jsonSchemaRDD2) - comparePlans(Schema(expectedSchema), Schema(unioned2.logicalPlan.output)) - } + jsonSchemaRDD.registerAsTable("jsonTable") + checkAnswer( + sql("select * from jsonTable"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + } } From 702763430f747ed1ba7fb2e25484bac7362dc4b6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 10:00:35 -0700 Subject: [PATCH 15/25] Java API. --- .../spark/sql/api/java/JavaSQLContext.scala | 18 +++++++- .../spark/sql/api/java/JavaSQLSuite.scala | 46 +++++++++++++++++++ 2 files changed, 63 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 6f7d431b9a819..af09d755cca16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.catalyst.types._ @@ -100,13 +101,28 @@ class JavaSQLContext(val sqlContext: SQLContext) { new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) } - /** * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. */ def parquetFile(path: String): JavaSchemaRDD = new JavaSchemaRDD(sqlContext, ParquetRelation(path)) + /** + * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. + * + * @group userf + */ + def jsonFile(path: String): JavaSchemaRDD = + jsonRDD(sqlContext.sparkContext.textFile(path)) + + /** + * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a + * [[JavaSchemaRDD]]. + * + * @group userf + */ + def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = + new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(json, 1.0)) /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 9fff7222fe840..f580818e899b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.api.java + import scala.beans.BeanProperty import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test.TestSQLContext // Implicits @@ -111,4 +113,48 @@ class JavaSQLSuite extends FunSuite { """.stripMargin).collect.head.row === Seq.fill(8)(null)) } + + test("loads JSON datasets") { + val jsonString = + """{"string":"this is a simple string.", + "integer":10, + "long":21474836470, + "bigInteger":92233720368547758070, + "double":1.7976931348623157E308, + "boolean":true, + "null":null + }""".replaceAll("\n", " ") + val rdd = javaCtx.parallelize(jsonString :: Nil) + + var schemaRDD = javaSqlCtx.jsonRDD(rdd) + + schemaRDD.registerAsTable("jsonTable1") + + assert( + javaSqlCtx.sql("select * from jsonTable1").collect.head.row === + Seq(BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.")) + + val file = getTempFilePath("json") + val path = file.toString + rdd.saveAsTextFile(path) + schemaRDD = javaSqlCtx.jsonFile(path) + + schemaRDD.registerAsTable("jsonTable2") + + assert( + javaSqlCtx.sql("select * from jsonTable2").collect.head.row === + Seq(BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.")) + } } From 9df8c5a6c474eca8274efceed261621b5ca7509b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 10:00:52 -0700 Subject: [PATCH 16/25] Python API. --- python/pyspark/sql.py | 57 ++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 56 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 960d0a82448aa..7d0678112cef4 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark.rdd import RDD +from pyspark.rdd import RDD, PipelinedRDD from py4j.protocol import Py4JError @@ -123,6 +123,49 @@ def parquetFile(self, path): jschema_rdd = self._ssql_ctx.parquetFile(path) return SchemaRDD(jschema_rdd, self) + + def jsonFile(self, path): + """Loads a text file storing one JSON object per line, + returning the result as a L{SchemaRDD}. + + >>> import tempfile, shutil + >>> jsonFile = tempfile.mkdtemp() + >>> shutil.rmtree(jsonFile) + >>> ofn = open(jsonFile, 'w') + >>> for json in jsonStrings: + ... print>>ofn, json + >>> ofn.close() + >>> srdd = sqlCtx.jsonFile(jsonFile) + >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, + ... {"f1" : 3, "f2": "row3"}] + True + """ + jschema_rdd = self._ssql_ctx.jsonFile(path) + return SchemaRDD(jschema_rdd, self) + + def jsonRDD(self, rdd): + """Loads a RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. + + >>> srdd = sqlCtx.jsonRDD(json) + >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, + ... {"f1" : 3, "f2": "row3"}] + True + """ + def func(split, iterator): + for x in iterator: + if not isinstance(x, basestring): + x = unicode(x) + yield x.encode("utf-8") + keyed = PipelinedRDD(rdd, func) + keyed._bypass_serializer = True + jrdd = keyed._jrdd.map(self._jvm.BytesToString()) + jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + return SchemaRDD(jschema_rdd, self) + def sql(self, sqlQuery): """Return a L{SchemaRDD} representing the result of the given query. @@ -323,6 +366,14 @@ def saveAsTable(self, tableName): """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) + def getSchemaTreeString(self): + """Returns the output schema in the tree format.""" + self._jschema_rdd.getSchemaTreeString() + + def printSchema(self): + """Prints out the schema in the tree format.""" + print self.getSchemaTreeString() + def count(self): """Return the number of elements in this RDD. @@ -420,6 +471,10 @@ def _test(): globs['sqlCtx'] = SQLContext(sc) globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) + jsonStrings = ['{"field1": 1, "field2": "row1"}', + '{"field1" : 2, "field2": "row2"}', '{"field1" : 3, "field2": "row3"}'] + globs['jsonStrings'] = jsonStrings + globs['json'] = sc.parallelize(jsonStrings) (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: From 4fbddf063e5f6bdebc2e1907fcfb70143bec5814 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 10:01:00 -0700 Subject: [PATCH 17/25] Programming guide. --- docs/sql-programming-guide.md | 374 ++++++++++++++++++++++------------ 1 file changed, 241 insertions(+), 133 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2b0c4c5414ff2..f0dc858a36084 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -17,20 +17,20 @@ Spark. At the core of this component is a new type of RDD, [Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) -file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). +file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`.
-Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using +Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, [JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed [Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) -file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). +file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
@@ -41,7 +41,7 @@ Spark. At the core of this component is a new type of RDD, [Row](api/python/pyspark.sql.Row-class.html) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) -file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). +file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell.
@@ -62,10 +62,10 @@ descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) +val sqlCtx = new org.apache.spark.sql.SQLContext(sc) -// Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import sqlContext._ +// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +import sqlCtx.createSchemaRDD {% endhighlight %} @@ -77,8 +77,8 @@ The entry point into all relational functionality in Spark is the of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} -JavaSparkContext ctx = ...; // An existing JavaSparkContext. -JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx); +JavaSparkContext sc = ...; // An existing JavaSparkContext. +JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc); {% endhighlight %} @@ -98,7 +98,9 @@ sqlCtx = SQLContext(sc) -## Running SQL on RDDs +# Data Sources + +## RDDs
@@ -111,8 +113,10 @@ types such as Sequences or Arrays. This RDD can be implicitly converted to a Sch registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -import sqlContext._ +// sc is an existing SparkContext. +val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +import sqlCtx.createSchemaRDD // Define the schema using a case class. // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, @@ -123,8 +127,8 @@ case class Person(name: String, age: Int) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) people.registerAsTable("people") -// SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +// SQL statements can be run by using the sql methods provided by sqlCtx. +val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -170,12 +174,11 @@ A schema can be applied to an existing RDD by calling `applySchema` and providin for the JavaBean. {% highlight java %} - -JavaSparkContext ctx = ...; // An existing JavaSparkContext. -JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx) +// sc is an existing JavaSparkContext. +JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc) // Load a text file and convert each line to a JavaBean. -JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( +JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( new Function() { public Person call(String line) throws Exception { String[] parts = line.split(","); @@ -215,6 +218,10 @@ row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as can be used in subsequent SQL statements. {% highlight python %} +# sc is an existing SparkContext. +from pyspark.sql import SQLContext +sqlCtx = SQLContext(sc) + # Load a text file and convert each line to a dictionary. lines = sc.textFile("examples/src/main/resources/people.txt") parts = lines.map(lambda l: l.split(",")) @@ -223,8 +230,8 @@ people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) # Infer the schema, and register the SchemaRDD as a table. # In future versions of PySpark we would like to add support for registering RDDs with other # datatypes as tables -peopleTable = sqlCtx.inferSchema(people) -peopleTable.registerAsTable("people") +schemaPeople = sqlCtx.inferSchema(people) +schemaPeople.registerAsTable("people") # SQL can be run over SchemaRDDs that have been registered as a table. teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -241,84 +248,7 @@ teenNames = teenagers.map(lambda p: "Name: " + p.name) Users that want a more complete dialect of SQL should look at the HiveQL support provided by `HiveContext`. -## Running SQL on JSON Datasets -**Running SQL on JSON datasets is currently only supported in Scala.** - -Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are three ways: - -1. Creating the SchemaRDD from text files that store one JSON object per line by using `jsonFile`. -2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object per string by using `jsonRDD`. -3. Creating the SchemaRDD by unioning two SchemaRDDs for two JSON datasets through `schemaRDD1.unionAll(schemaRDD2)`. - -The schema (field names and types) of a JSON dataset is automatically inferred when the SchemaRDD is created. - -{% highlight scala %} -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -import sqlContext._ - -// A JSON dataset is pointed by path. -// The path can be either a single text file or a directory storing text files. -val path = "examples/src/main/resources/people.json" -// Create a SchemaRDD from the file(s) pointed by path -val people = jsonFile(path) - -// Because the schema of a JSON dataset is automatically inferred, to write queries, -// it is better to take a look at what is the schema. -people.printSchema() -// The schema of people is ... -// root -// |-- age: IntegerType -// |-- name: StringType - -// Register this SchemaRDD as a table. -people.registerAsTable("people") - -// SQL statements can be run by using the sql methods provided by sqlContext. -val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) - -// Let's create another SchemaRDD from a RDD[String] storing one JSON object per string. -val anotherPeopleRDD = sc.parallelize( - """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) -val anotherPeople = jsonRDD(anotherPeopleRDD) - -// Take a look at the schema of this new SchemaRDD. -anotherPeople.printSchema() -// The schema of anotherPeople is ... -// root -// |-- address: StructType -// | |-- city: StringType -// | |-- state: StringType -// |-- name: StringType - -// Two SchemaRDDs created from JSON datasets can be unioned together. -val peopleUnioned = people.unionAll(anotherPeople) - -// Take a look at the schema of this unioned SchemaRDD. -peopleUnioned.printSchema() -// The schema of this unioned SchemaRDD is ... -// root -// |-- address: StructType -// | |-- city: StringType -// | |-- state: StringType -// |-- age: IntegerType -// |-- name: StringType - -// Register this unioned SchemaRDD as a table. -peopleUnioned.registerAsTable("peopleUnioned") -val peopleFromColumbus = - sql("SELECT name, address.city FROM peopleUnioned WHERE address.city = 'Columbus'") - -// See the result. -peopleFromColumbus.map(t => "Name: " + t(0) + ", City: " + t(1)).collect().foreach(println) - -{% endhighlight %} - - -## Using Parquet +## Parquet Files [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema @@ -329,21 +259,22 @@ of the original data. Using the data from the above example:
{% highlight scala %} -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -import sqlContext._ +// sqlCtx from the previous example is used in this example. +// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +import sqlCtx.createSchemaRDD val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. -// The RDD is implicitly converted to a SchemaRDD, allowing it to be stored using Parquet. +// The RDD is implicitly converted to a SchemaRDD by createSchemaRDD, allowing it to be stored using Parquet. people.saveAsParquetFile("people.parquet") // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. -// The result of loading a Parquet file is also a JavaSchemaRDD. -val parquetFile = sqlContext.parquetFile("people.parquet") +// The result of loading a Parquet file is also a SchemaRDD. +val parquetFile = sqlCtx.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile") -val teenagers = sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +val teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.collect().foreach(println) {% endhighlight %} @@ -352,6 +283,7 @@ teenagers.collect().foreach(println)
{% highlight java %} +// sqlCtx from the previous example is used in this example. JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example. @@ -365,8 +297,11 @@ JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile"); JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); - - +List teenagerNames = teenagers.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); {% endhighlight %}
@@ -374,6 +309,7 @@ JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >=
{% highlight python %} +# sqlCtx from the previous example is used in this example. peopleTable # The SchemaRDD from the previous example. @@ -387,37 +323,186 @@ parquetFile = sqlCtx.parquetFile("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile"); teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") - +teenNames = teenagers.map(lambda p: "Name: " + p.name) {% endhighlight %}
-## Writing Language-Integrated Relational Queries +## JSON Datasets +
-**Language-Integrated queries are currently only supported in Scala.** +
+Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are two ways to create a SchemaRDD for a JSON dataset: -Spark SQL also supports a domain specific language for writing queries. Once again, -using the data from the above examples: +1. Creating the SchemaRDD from text files that store one JSON object per line. +2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. + +The schema of a JSON dataset is automatically inferred when the SchemaRDD is created. {% highlight scala %} -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -import sqlContext._ -val people: RDD[Person] = ... // An RDD of case class objects, from the first example. +// sc is an existing SparkContext. +val sqlCtx = new org.apache.spark.sql.SQLContext(sc) -// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' -val teenagers = people.where('age >= 10).where('age <= 19).select('name) +// A JSON dataset is pointed by path. +// The path can be either a single text file or a directory storing text files. +val path = "examples/src/main/resources/people.json" +// Create a SchemaRDD from the file(s) pointed by path +val people = sqlCtx.jsonFile(path) + +// Because the schema of a JSON dataset is automatically inferred, to write queries, +// it is better to take a look at what is the schema. +people.printSchema() +// The schema of people is ... +// root +// |-- age: IntegerType +// |-- name: StringType + +// Register this SchemaRDD as a table. +people.registerAsTable("people") + +// SQL statements can be run by using the sql methods provided by sqlCtx. +val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) + +// Alternatively, a SchemaRDD can be created for a JSON dataset represented by +// a RDD[String] storing one JSON object per string. +val anotherPeopleRDD = sc.parallelize( + """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) +val anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) + +// Take a look at the schema of this new SchemaRDD. +anotherPeople.printSchema() +// The schema of anotherPeople is ... +// root +// |-- address: StructType +// | |-- city: StringType +// | |-- state: StringType +// |-- name: StringType {% endhighlight %} -The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers -prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are -evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). +
+ +
+Spark SQL supports querying JSON datasets. To query a JSON dataset, a JavaSchemaRDD needs to be created for this JSON dataset. There are two ways to create a JavaSchemaRDD for a JSON dataset: + +1. Creating the JavaSchemaRDD from text files that store one JSON object per line. +2. Creating the JavaSchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. - +The schema of a JSON dataset is automatically inferred when the JavaSchemaRDD is created. -# Hive Support +{% highlight java %} +// sc is an existing JavaSparkContext. +JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc); + +// A JSON dataset is pointed by path. +// The path can be either a single text file or a directory storing text files. +String path = "examples/src/main/resources/people.json" +// Create a JavaSchemaRDD from the file(s) pointed by path +JavaSchemaRDD people = sqlCtx.jsonFile(path) + +// Because the schema of a JSON dataset is automatically inferred, to write queries, +// it is better to take a look at what is the schema. +people.printSchema() +// The schema of people is ... +// root +// |-- age: IntegerType +// |-- name: StringType + +// Register this JavaSchemaRDD as a table. +people.registerAsTable("people") + +// SQL statements can be run by using the sql methods provided by sqlCtx. +JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +// The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +List teenagerNames = teenagers.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); + +// Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by +// a RDD[String] storing one JSON object per string. +List jsonData = Arrays.asList( + "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); +JavaRDD anotherPeopleRDD = sc.parallelize(jsonData) +val anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) + +// Take a look at the schema of this new JavaSchemaRDD. +anotherPeople.printSchema() +// The schema of anotherPeople is ... +// root +// |-- address: StructType +// | |-- city: StringType +// | |-- state: StringType +// |-- name: StringType + +{% endhighlight %} +
+ +
+Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are two ways to create a SchemaRDD for a JSON dataset: + +1. Creating the SchemaRDD from text files that store one JSON object per line. +2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. + +The schema of a JSON dataset is automatically inferred when the SchemaRDD is created. + +{% highlight python %} +# sc is an existing SparkContext. +from pyspark.sql import SQLContext +sqlCtx = SQLContext(sc) + +# A JSON dataset is pointed by path. +# The path can be either a single text file or a directory storing text files. +path = "examples/src/main/resources/people.json" +# Create a SchemaRDD from the file(s) pointed by path +people = sqlCtx.jsonFile(path) + +# Because the schema of a JSON dataset is automatically inferred, to write queries, +# it is better to take a look at what is the schema. +people.printSchema() +# The schema of people is ... +# root +# |-- age: IntegerType +# |-- name: StringType + +# Register this SchemaRDD as a table. +people.registerAsTable("people") + +# SQL statements can be run by using the sql methods provided by sqlCtx. +val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +# The columns of a row in the result can be accessed by ordinal. +teenNames = teenagers.map(lambda p: "Name: " + p.name) + +# Alternatively, a SchemaRDD can be created for a JSON dataset represented by +# a RDD[String] storing one JSON object per string. +anotherPeopleRDD = sc.parallelize([ + '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) +anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) + +# Take a look at the schema of this new SchemaRDD. +anotherPeople.printSchema() +# The schema of anotherPeople is ... +# root +# |-- address: StructType +# | |-- city: StringType +# | |-- state: StringType +# |-- name: StringType +{% endhighlight %} +
+ +
+ +## Hive Tables Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. @@ -439,17 +524,14 @@ which is similar to `HiveContext`, but creates a local copy of the `metastore` a automatically. {% highlight scala %} -val sc: SparkContext // An existing SparkContext. -val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) - -// Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import hiveContext._ +// sc is an existing SparkContext. +val hiveCtx = new org.apache.spark.sql.hive.HiveContext(sc) -hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hql("FROM src SELECT key, value").collect().foreach(println) +hiveCtx.hql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
@@ -462,8 +544,8 @@ the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allow expressed in HiveQL. {% highlight java %} -JavaSparkContext ctx = ...; // An existing JavaSparkContext. -JavaHiveContext hiveCtx = new org.apache.spark.sql.hive.api.java.HiveContext(ctx); +// sc is an existing JavaSparkContext. +JavaHiveContext hiveCtx = new org.apache.spark.sql.hive.api.java.HiveContext(sc); hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); @@ -483,7 +565,7 @@ the `sql` method a `HiveContext` also provides an `hql` methods, which allows qu expressed in HiveQL. {% highlight python %} - +# sc is an existing SparkContext. from pyspark.sql import HiveContext hiveCtx = HiveContext(sc) @@ -497,3 +579,29 @@ results = hiveCtx.hql("FROM src SELECT key, value").collect() + + +# Writing Language-Integrated Relational Queries + +**Language-Integrated queries are currently only supported in Scala.** + +Spark SQL also supports a domain specific language for writing queries. Once again, +using the data from the above examples: + +{% highlight scala %} +// sc is an existing SparkContext. +val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +// Importing the SQL context gives access to all the public SQL functions and implicit conversions. +import sqlCtx._ +val people: RDD[Person] = ... // An RDD of case class objects, from the first example. + +// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' +val teenagers = people.where('age >= 10).where('age <= 19).select('name) +{% endhighlight %} + +The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers +prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are +evaluated by the SQL execution engine. A full list of the functions supported can be found in the +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). + + \ No newline at end of file From e7a6c19308aba0033fa7df1a700f8b310b59a55e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 11:48:24 -0700 Subject: [PATCH 18/25] SchemaRDD.javaToPython should convert a field with the StructType to a Map. --- docs/sql-programming-guide.md | 15 +++++++--- python/pyspark/sql.py | 21 ++++++++------ .../org/apache/spark/sql/SchemaRDD.scala | 29 ++++++++++++++++--- 3 files changed, 48 insertions(+), 17 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f0dc858a36084..6d725a0c42e80 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -238,6 +238,8 @@ teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # The results of SQL queries are RDDs and support all the normal RDD operations. teenNames = teenagers.map(lambda p: "Name: " + p.name) +for teenName in teenNames.collect(): + print teenName {% endhighlight %} @@ -275,7 +277,7 @@ val parquetFile = sqlCtx.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile") val teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") -teenagers.collect().foreach(println) +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -311,10 +313,10 @@ List teenagerNames = teenagers.map(new Function() { {% highlight python %} # sqlCtx from the previous example is used in this example. -peopleTable # The SchemaRDD from the previous example. +schemaPeople # The SchemaRDD from the previous example. # SchemaRDDs can be saved as Parquet files, maintaining the schema information. -peopleTable.saveAsParquetFile("people.parquet") +schemaPeople.saveAsParquetFile("people.parquet") # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. # The result of loading a parquet file is also a SchemaRDD. @@ -324,6 +326,8 @@ parquetFile = sqlCtx.parquetFile("people.parquet") parquetFile.registerAsTable("parquetFile"); teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) +for teenName in teenNames.collect(): + print teenName {% endhighlight %} @@ -477,11 +481,13 @@ people.printSchema() people.registerAsTable("people") # SQL statements can be run by using the sql methods provided by sqlCtx. -val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # The results of SQL queries are SchemaRDDs and support all the normal RDD operations. # The columns of a row in the result can be accessed by ordinal. teenNames = teenagers.map(lambda p: "Name: " + p.name) +for teenName in teenNames.collect(): + print teenName # Alternatively, a SchemaRDD can be created for a JSON dataset represented by # a RDD[String] storing one JSON object per string. @@ -597,6 +603,7 @@ val people: RDD[Person] = ... // An RDD of case class objects, from the first ex // The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' val teenagers = people.where('age >= 10).where('age <= 19).select('name) +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 1ab762866382f..9041668f33069 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -138,9 +138,10 @@ def jsonFile(self, path): >>> ofn.close() >>> srdd = sqlCtx.jsonFile(jsonFile) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") - >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, - ... {"f1" : 3, "f2": "row3"}] + >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") + >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, + ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, + ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] True """ jschema_rdd = self._ssql_ctx.jsonFile(path) @@ -151,9 +152,10 @@ def jsonRDD(self, rdd): >>> srdd = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd, "table1") - >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") - >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, - ... {"f1" : 3, "f2": "row3"}] + >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2, field3 as f3 from table1") + >>> srdd2.collect() == [{"f1": 1, "f2": "row1", "f3":{"field4":11}}, + ... {"f1": 2, "f2": "row2", "f3":{"field4":22}}, + ... {"f1": 3, "f2": "row3", "f3":{"field4":33}}] True """ def func(split, iterator): @@ -369,7 +371,7 @@ def saveAsTable(self, tableName): def getSchemaTreeString(self): """Returns the output schema in the tree format.""" - self._jschema_rdd.getSchemaTreeString() + return self._jschema_rdd.getSchemaTreeString() def printSchema(self): """Prints out the schema in the tree format.""" @@ -473,8 +475,9 @@ def _test(): globs['sqlCtx'] = SQLContext(sc) globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) - jsonStrings = ['{"field1": 1, "field2": "row1"}', - '{"field1" : 2, "field2": "row2"}', '{"field1" : 3, "field2": "row3"}'] + jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}', + '{"field1" : 2, "field2": "row2", "field3":{"field4":22}}', + '{"field1" : 3, "field2": "row3", "field3":{"field4":33}}'] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 42ec8f6104a67..bcbeee31681b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.catalyst.types.{DataType, StructType, BooleanType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD import java.util.{Map => JMap} @@ -344,13 +344,34 @@ class SchemaRDD( def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) private[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val fieldNames: Seq[String] = this.queryExecution.analyzed.output.map(_.name) + def rowToMap(row: Row, structType: StructType): JMap[String, Any] = { + val fields = structType.fields.map(field => (field.name, field.dataType)) + val map: JMap[String, Any] = new java.util.HashMap + row.zip(fields).foreach { + case (obj, (name, dataType)) => + dataType match { + case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) + case other => map.put(name, obj) + } + } + + map + } + + // TODO: Actually, the schema of a row should be represented by a StructType instead of + // a Seq[Attribute]. Once we have finished that change, we can just use rowToMap to + // construct the Map for python. + val fields: Seq[(String, DataType)] = this.queryExecution.analyzed.output.map( + field => (field.name, field.dataType)) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => val map: JMap[String, Any] = new java.util.HashMap - row.zip(fieldNames).foreach { case (obj, name) => - map.put(name, obj) + row.zip(fields).foreach { case (obj, (name, dataType)) => + dataType match { + case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) + case other => map.put(name, obj) + } } map }.grouped(10).map(batched => pickle.dumps(batched.toArray)) From 83013fb9e7f332007ee3c69920d6eb20892b293b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 12:14:42 -0700 Subject: [PATCH 19/25] Update Java Example. --- docs/sql-programming-guide.md | 16 ++-- .../spark/examples/sql/JavaSparkSQL.java | 79 ++++++++++++++++++- 2 files changed, 85 insertions(+), 10 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6d725a0c42e80..d195cea88fb83 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -405,23 +405,23 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc); // A JSON dataset is pointed by path. // The path can be either a single text file or a directory storing text files. -String path = "examples/src/main/resources/people.json" +String path = "examples/src/main/resources/people.json"; // Create a JavaSchemaRDD from the file(s) pointed by path -JavaSchemaRDD people = sqlCtx.jsonFile(path) +JavaSchemaRDD people = sqlCtx.jsonFile(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. -people.printSchema() +people.printSchema(); // The schema of people is ... // root // |-- age: IntegerType // |-- name: StringType // Register this JavaSchemaRDD as a table. -people.registerAsTable("people") +people.registerAsTable("people"); // SQL statements can be run by using the sql methods provided by sqlCtx. -JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -435,11 +435,11 @@ List teenagerNames = teenagers.map(new Function() { // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); -JavaRDD anotherPeopleRDD = sc.parallelize(jsonData) -val anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) +JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); +JavaSchemaRDD anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD); // Take a look at the schema of this new JavaSchemaRDD. -anotherPeople.printSchema() +anotherPeople.printSchema(); // The schema of anotherPeople is ... // root // |-- address: StructType diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index ad5ec84b71e69..4a75f965cd170 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -18,6 +18,7 @@ package org.apache.spark.examples.sql; import java.io.Serializable; +import java.util.Arrays; import java.util.List; import org.apache.spark.SparkConf; @@ -56,6 +57,8 @@ public static void main(String[] args) throws Exception { JavaSparkContext ctx = new JavaSparkContext(sparkConf); JavaSQLContext sqlCtx = new JavaSQLContext(ctx); + + System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( new Function() { @@ -84,16 +87,88 @@ public String call(Row row) { return "Name: " + row.getString(0); } }).collect(); + for (String name: teenagerNames) { + System.out.println(name); + } + System.out.println("=== Data source: Parquet File ==="); // JavaSchemaRDDs can be saved as parquet files, maintaining the schema information. schemaPeople.saveAsParquetFile("people.parquet"); - // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. + // Read in the parquet file created above. + // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a JavaSchemaRDD. JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile"); - JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + JavaSchemaRDD teenagers2 = + sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + teenagerNames = teenagers2.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } + }).collect(); + for (String name: teenagerNames) { + System.out.println(name); + } + + System.out.println("=== Data source: JSON Dataset ==="); + // A JSON dataset is pointed by path. + // The path can be either a single text file or a directory storing text files. + String path = "examples/src/main/resources/people.json"; + // Create a JavaSchemaRDD from the file(s) pointed by path + JavaSchemaRDD peopleFromJsonFile = sqlCtx.jsonFile(path); + + // Because the schema of a JSON dataset is automatically inferred, to write queries, + // it is better to take a look at what is the schema. + peopleFromJsonFile.printSchema(); + // The schema of people is ... + // root + // |-- age: IntegerType + // |-- name: StringType + + // Register this JavaSchemaRDD as a table. + peopleFromJsonFile.registerAsTable("people"); + + // SQL statements can be run by using the sql methods provided by sqlCtx. + JavaSchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + + // The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. + // The columns of a row in the result can be accessed by ordinal. + teenagerNames = teenagers3.map(new Function() { + public String call(Row row) { return "Name: " + row.getString(0); } + }).collect(); + for (String name: teenagerNames) { + System.out.println(name); + } + + // Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by + // a RDD[String] storing one JSON object per string. + List jsonData = Arrays.asList( + "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); + JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); + JavaSchemaRDD peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD); + + // Take a look at the schema of this new JavaSchemaRDD. + peopleFromJsonRDD.printSchema(); + // The schema of anotherPeople is ... + // root + // |-- address: StructType + // | |-- city: StringType + // | |-- state: StringType + // |-- name: StringType + + peopleFromJsonRDD.registerAsTable("people2"); + + JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); + List nameAndCity = peopleWithCity.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0) + ", City: " + row.getString(1); + } + }).collect(); + for (String name: nameAndCity) { + System.out.println(name); + } } } From 7ea750ec23c796504f297fc4f1f61fb8b2b1a761 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 16:04:14 -0700 Subject: [PATCH 20/25] marmbrus's comments. --- .rat-excludes | 1 + docs/sql-programming-guide.md | 205 ++++++++---------- python/pyspark/sql.py | 2 +- .../catalyst/analysis/HiveTypeCoercion.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 50 ++++- .../spark/sql/catalyst/plans/package.scala | 49 +---- .../optimizer/CombiningLimitsSuite.scala | 3 +- .../optimizer/ConstantFoldingSuite.scala | 3 +- .../optimizer/FilterPushdownSuite.scala | 5 +- .../catalyst/optimizer/OptimizerTest.scala | 26 --- ...mplifyCaseConversionExpressionsSuite.scala | 3 +- .../org/apache/spark/sql/SQLContext.scala | 2 + .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../org/apache/spark/sql/json/JsonRDD.scala | 7 +- .../spark/sql/api/java/JavaSQLSuite.scala | 1 - 15 files changed, 151 insertions(+), 214 deletions(-) delete mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala diff --git a/.rat-excludes b/.rat-excludes index 52b2dfac5cf2b..15344dfb292db 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -22,6 +22,7 @@ spark-env.sh.template log4j-defaults.properties sorttable.js .*txt +.*json .*data .*log cloudpickle.py diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d195cea88fb83..cd953a514148f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -62,10 +62,10 @@ descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. -val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +val sqlContext = new org.apache.spark.sql.SQLContext(sc) // createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. -import sqlCtx.createSchemaRDD +import sqlContext.createSchemaRDD {% endhighlight %} @@ -78,7 +78,7 @@ of its descendants. To create a basic JavaSQLContext, all you need is a JavaSpa {% highlight java %} JavaSparkContext sc = ...; // An existing JavaSparkContext. -JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); {% endhighlight %} @@ -91,7 +91,7 @@ of its decedents. To create a basic SQLContext, all you need is a SparkContext. {% highlight python %} from pyspark.sql import SQLContext -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) {% endhighlight %} @@ -100,6 +100,23 @@ sqlCtx = SQLContext(sc) # Data Sources +
+
+Spark SQL supports operating of a variety of data sources though the SchemaRDD interface. +Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. +
+ +
+Spark SQL supports operating of a variety of data sources though the JavaSchemaRDD interface. +Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. +
+ +
+Spark SQL supports operating of a variety of data sources though the SchemaRDD interface. +Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. +
+
+ ## RDDs
@@ -114,9 +131,9 @@ registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} // sc is an existing SparkContext. -val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +val sqlContext = new org.apache.spark.sql.SQLContext(sc) // createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. -import sqlCtx.createSchemaRDD +import sqlContext.createSchemaRDD // Define the schema using a case class. // Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, @@ -127,8 +144,8 @@ case class Person(name: String, age: Int) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) people.registerAsTable("people") -// SQL statements can be run by using the sql methods provided by sqlCtx. -val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +// SQL statements can be run by using the sql methods provided by sqlContext. +val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -175,7 +192,7 @@ for the JavaBean. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc) +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc) // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( @@ -192,11 +209,11 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m }); // Apply a schema to an RDD of JavaBeans and register it as a table. -JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); +JavaSchemaRDD schemaPeople = sqlContext.applySchema(people, Person.class); schemaPeople.registerAsTable("people"); // SQL can be run over RDDs that have been registered as tables. -JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -220,7 +237,7 @@ can be used in subsequent SQL statements. {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import SQLContext -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) # Load a text file and convert each line to a dictionary. lines = sc.textFile("examples/src/main/resources/people.txt") @@ -230,11 +247,11 @@ people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) # Infer the schema, and register the SchemaRDD as a table. # In future versions of PySpark we would like to add support for registering RDDs with other # datatypes as tables -schemaPeople = sqlCtx.inferSchema(people) +schemaPeople = sqlContext.inferSchema(people) schemaPeople.registerAsTable("people") # SQL can be run over SchemaRDDs that have been registered as a table. -teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # The results of SQL queries are RDDs and support all the normal RDD operations. teenNames = teenagers.map(lambda p: "Name: " + p.name) @@ -261,9 +278,9 @@ of the original data. Using the data from the above example:
{% highlight scala %} -// sqlCtx from the previous example is used in this example. +// sqlContext from the previous example is used in this example. // createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. -import sqlCtx.createSchemaRDD +import sqlContext.createSchemaRDD val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. @@ -272,11 +289,11 @@ people.saveAsParquetFile("people.parquet") // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. // The result of loading a Parquet file is also a SchemaRDD. -val parquetFile = sqlCtx.parquetFile("people.parquet") +val parquetFile = sqlContext.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile") -val teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -285,7 +302,7 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
{% highlight java %} -// sqlCtx from the previous example is used in this example. +// sqlContext from the previous example is used in this example. JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example. @@ -294,11 +311,11 @@ schemaPeople.saveAsParquetFile("people.parquet"); // Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a JavaSchemaRDD. -JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); +JavaSchemaRDD parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile"); -JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.map(new Function() { public String call(Row row) { return "Name: " + row.getString(0); @@ -311,7 +328,7 @@ List teenagerNames = teenagers.map(new Function() {
{% highlight python %} -# sqlCtx from the previous example is used in this example. +# sqlContext from the previous example is used in this example. schemaPeople # The SchemaRDD from the previous example. @@ -320,11 +337,11 @@ schemaPeople.saveAsParquetFile("people.parquet") # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. # The result of loading a parquet file is also a SchemaRDD. -parquetFile = sqlCtx.parquetFile("people.parquet") +parquetFile = sqlContext.parquetFile("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerAsTable("parquetFile"); -teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): print teenName @@ -338,25 +355,23 @@ for teenName in teenNames.collect():
-Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are two ways to create a SchemaRDD for a JSON dataset: - -1. Creating the SchemaRDD from text files that store one JSON object per line. -2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. +Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD. +This conversion can be done using one of two methods in a SQLContext: -The schema of a JSON dataset is automatically inferred when the SchemaRDD is created. +* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. +* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight scala %} // sc is an existing SparkContext. -val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +val sqlContext = new org.apache.spark.sql.SQLContext(sc) // A JSON dataset is pointed by path. // The path can be either a single text file or a directory storing text files. val path = "examples/src/main/resources/people.json" // Create a SchemaRDD from the file(s) pointed by path -val people = sqlCtx.jsonFile(path) +val people = sqlContext.jsonFile(path) -// Because the schema of a JSON dataset is automatically inferred, to write queries, -// it is better to take a look at what is the schema. +// The inferred schema can be visualized using the printSchema() method. people.printSchema() // The schema of people is ... // root @@ -366,51 +381,36 @@ people.printSchema() // Register this SchemaRDD as a table. people.registerAsTable("people") -// SQL statements can be run by using the sql methods provided by sqlCtx. -val teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +// SQL statements can be run by using the sql methods provided by sqlContext. +val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // Alternatively, a SchemaRDD can be created for a JSON dataset represented by // a RDD[String] storing one JSON object per string. val anotherPeopleRDD = sc.parallelize( """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) -val anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) - -// Take a look at the schema of this new SchemaRDD. -anotherPeople.printSchema() -// The schema of anotherPeople is ... -// root -// |-- address: StructType -// | |-- city: StringType -// | |-- state: StringType -// |-- name: StringType +val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) {% endhighlight %}
-Spark SQL supports querying JSON datasets. To query a JSON dataset, a JavaSchemaRDD needs to be created for this JSON dataset. There are two ways to create a JavaSchemaRDD for a JSON dataset: - -1. Creating the JavaSchemaRDD from text files that store one JSON object per line. -2. Creating the JavaSchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. +Spark SQL can automatically infer the schema of a JSON dataset and load it as a JavaSchemaRDD. +This conversion can be done using one of two methods in a JavaSQLContext : -The schema of a JSON dataset is automatically inferred when the JavaSchemaRDD is created. +* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. +* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(sc); +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); // A JSON dataset is pointed by path. // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a JavaSchemaRDD from the file(s) pointed by path -JavaSchemaRDD people = sqlCtx.jsonFile(path); +JavaSchemaRDD people = sqlContext.jsonFile(path); -// Because the schema of a JSON dataset is automatically inferred, to write queries, -// it is better to take a look at what is the schema. +// The inferred schema can be visualized using the printSchema() method. people.printSchema(); // The schema of people is ... // root @@ -420,57 +420,37 @@ people.printSchema(); // Register this JavaSchemaRDD as a table. people.registerAsTable("people"); -// SQL statements can be run by using the sql methods provided by sqlCtx. -JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); - -// The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. -// The columns of a row in the result can be accessed by ordinal. -List teenagerNames = teenagers.map(new Function() { - public String call(Row row) { - return "Name: " + row.getString(0); - } -}).collect(); +// SQL statements can be run by using the sql methods provided by sqlContext. +JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by // a RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); -JavaSchemaRDD anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD); - -// Take a look at the schema of this new JavaSchemaRDD. -anotherPeople.printSchema(); -// The schema of anotherPeople is ... -// root -// |-- address: StructType -// | |-- city: StringType -// | |-- state: StringType -// |-- name: StringType - +JavaSchemaRDD anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD); {% endhighlight %}
-Spark SQL supports querying JSON datasets. To query a JSON dataset, a SchemaRDD needs to be created for this JSON dataset. There are two ways to create a SchemaRDD for a JSON dataset: - -1. Creating the SchemaRDD from text files that store one JSON object per line. -2. Creating the SchemaRDD from a RDD of strings (`RDD[String]`) that stores one JSON object. +Spark SQL can automatically infer the schema of a JSON dataset and load it as a SchemaRDD. +This conversion can be done using one of two methods in a SQLContext: -The schema of a JSON dataset is automatically inferred when the SchemaRDD is created. +* `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. +* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import SQLContext -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) # A JSON dataset is pointed by path. # The path can be either a single text file or a directory storing text files. path = "examples/src/main/resources/people.json" # Create a SchemaRDD from the file(s) pointed by path -people = sqlCtx.jsonFile(path) +people = sqlContext.jsonFile(path) -# Because the schema of a JSON dataset is automatically inferred, to write queries, -# it is better to take a look at what is the schema. +# The inferred schema can be visualized using the printSchema() method. people.printSchema() # The schema of people is ... # root @@ -480,29 +460,14 @@ people.printSchema() # Register this SchemaRDD as a table. people.registerAsTable("people") -# SQL statements can be run by using the sql methods provided by sqlCtx. -teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") - -# The results of SQL queries are SchemaRDDs and support all the normal RDD operations. -# The columns of a row in the result can be accessed by ordinal. -teenNames = teenagers.map(lambda p: "Name: " + p.name) -for teenName in teenNames.collect(): - print teenName +# SQL statements can be run by using the sql methods provided by sqlContext. +teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a SchemaRDD can be created for a JSON dataset represented by # a RDD[String] storing one JSON object per string. anotherPeopleRDD = sc.parallelize([ '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) -anotherPeople = sqlCtx.jsonRDD(anotherPeopleRDD) - -# Take a look at the schema of this new SchemaRDD. -anotherPeople.printSchema() -# The schema of anotherPeople is ... -# root -# |-- address: StructType -# | |-- city: StringType -# | |-- state: StringType -# |-- name: StringType +anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) {% endhighlight %}
@@ -531,13 +496,13 @@ automatically. {% highlight scala %} // sc is an existing SparkContext. -val hiveCtx = new org.apache.spark.sql.hive.HiveContext(sc) +val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) -hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hiveCtx.hql("FROM src SELECT key, value").collect().foreach(println) +hiveContext.hql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
@@ -551,13 +516,13 @@ expressed in HiveQL. {% highlight java %} // sc is an existing JavaSparkContext. -JavaHiveContext hiveCtx = new org.apache.spark.sql.hive.api.java.HiveContext(sc); +JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); -hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect(); +Row[] results = hiveContext.hql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -573,13 +538,13 @@ expressed in HiveQL. {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import HiveContext -hiveCtx = HiveContext(sc) +hiveContext = HiveContext(sc) -hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = hiveCtx.hql("FROM src SELECT key, value").collect() +results = hiveContext.hql("FROM src SELECT key, value").collect() {% endhighlight %} @@ -596,9 +561,9 @@ using the data from the above examples: {% highlight scala %} // sc is an existing SparkContext. -val sqlCtx = new org.apache.spark.sql.SQLContext(sc) +val sqlContext = new org.apache.spark.sql.SQLContext(sc) // Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import sqlCtx._ +import sqlContext._ val people: RDD[Person] = ... // An RDD of case class objects, from the first example. // The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 12d3d05af02fe..e511f63769cf0 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -384,7 +384,7 @@ def saveAsTable(self, tableName): def getSchemaTreeString(self): """Returns the output schema in the tree format.""" - return self._jschema_rdd.getSchemaTreeString() + return self._jschema_rdd.schemaTreeString() def printSchema(self): """Prints out the schema in the tree format.""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 3e38a04882902..99fb8d8ff9b48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -128,8 +128,8 @@ trait HiveTypeCoercion { def findTightestCommonType(t1: DataType, t2: DataType): Option[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)) + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) // If found return the widest common type, otherwise None applicableConversion.map(_.filter(t => t == t1 || t == t2).last) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 9b34f6e0bbf31..8452eb98a3be7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.plans +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => @@ -125,9 +126,52 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } + protected def generateSchemaTreeString(schema: Seq[Attribute]): String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + schema.foreach { attribute => + val name = attribute.name + val dataType = attribute.dataType + dataType match { + case fields: StructType => + builder.append(s"$prefix-- $name: $StructType\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case ArrayType(fields: StructType) => + builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case ArrayType(elementType: DataType) => + builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") + case _ => builder.append(s"$prefix-- $name: $dataType\n") + } + } + + builder.toString() + } + + protected def generateSchemaTreeString( + schema: StructType, + prefix: String, + builder: StringBuilder): StringBuilder = { + schema.fields.foreach { + case StructField(name, fields: StructType, _) => + builder.append(s"$prefix-- $name: $StructType\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case StructField(name, ArrayType(fields: StructType), _) => + builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") + generateSchemaTreeString(fields, s"$prefix |", builder) + case StructField(name, ArrayType(elementType: DataType), _) => + builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") + case StructField(name, fieldType: DataType, _) => + builder.append(s"$prefix-- $name: $fieldType\n") + } + + builder + } + /** Returns the output schema in the tree format. */ - def getSchemaTreeString(): String = plans.generateSchemaTreeString(output) + def schemaTreeString: String = generateSchemaTreeString(output) /** Prints out the schema in the tree format */ - def printSchema(): Unit = println(getSchemaTreeString()) + def printSchema(): Unit = println(schemaTreeString) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 9a1571a682cb4..615d5739c8f3b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -17,55 +17,8 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} - /** * A a collection of common abstractions for query plans as well as * a base logical plan representation. */ -package object plans { - - def generateSchemaTreeString(schema: Seq[Attribute]): String = { - val builder = new StringBuilder - builder.append("root\n") - val prefix = " |" - schema.foreach { attribute => - val name = attribute.name - val dataType = attribute.dataType - dataType match { - case fields: StructType => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaTreeString(fields, s"$prefix |", builder) - case ArrayType(fields: StructType) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaTreeString(fields, s"$prefix |", builder) - case ArrayType(elementType: DataType) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case _ => builder.append(s"$prefix-- $name: $dataType\n") - } - } - - builder.toString() - } - - def generateSchemaTreeString( - schema: StructType, - prefix: String, - builder: StringBuilder): StringBuilder = { - schema.fields.foreach { - case StructField(name, fields: StructType, _) => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaTreeString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(fields: StructType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaTreeString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(elementType: DataType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case StructField(name, fieldType: DataType, _) => - builder.append(s"$prefix-- $name: $fieldType\n") - } - - builder - } -} +package object plans \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 714f01843c0f5..4896f1b955f01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ -class CombiningLimitsSuite extends OptimizerTest { +class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 6efc0e211eb21..cea97c584f7e1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types._ @@ -27,7 +28,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ -class ConstantFoldingSuite extends OptimizerTest { +class ConstantFoldingSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 1f67c80e54906..ebb123c1f909e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.plans.LeftOuter -import org.apache.spark.sql.catalyst.plans.RightOuter +import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ -class FilterPushdownSuite extends OptimizerTest { +class FilterPushdownSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala deleted file mode 100644 index 316b5e3bc5d41..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.optimizer - -import org.apache.spark.sql.catalyst.plans.PlanTest - -/** - * Provides helper methods for comparing plans produced by optimization rules with the expected - * result - */ -class OptimizerTest extends PlanTest diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala index df1409fe7baee..22992fb6f50d4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyCaseConversionExpressionsSuite.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.rules._ /* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -class SimplifyCaseConversionExpressionsSuite extends OptimizerTest { +class SimplifyCaseConversionExpressionsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 8e87aaf62b913..9d72c212538d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -106,6 +106,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: */ + @Experimental def jsonFile(path: String, samplingRatio: Double): SchemaRDD = { val json = sparkContext.textFile(path) jsonRDD(json, samplingRatio) @@ -122,6 +123,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * :: Experimental :: */ + @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = new SchemaRDD(this, JsonRDD.inferSchema(json, samplingRatio)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 206063ff53c35..2b722547dbd0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -124,8 +124,8 @@ private[sql] trait SchemaRDDLike { sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd /** Returns the output schema in the tree format. */ - def getSchemaTreeString(): String = queryExecution.analyzed.getSchemaTreeString() + def schemaTreeString: String = queryExecution.analyzed.schemaTreeString /** Prints out the schema in the tree format. */ - def printSchema(): Unit = println(getSchemaTreeString()) + def printSchema(): Unit = println(schemaTreeString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index ca912d33e4548..5d24f8ecf2866 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -22,7 +22,6 @@ 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._ @@ -31,10 +30,8 @@ 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 = { @@ -263,8 +260,8 @@ private[sql] object JsonRDD extends Logging { // the ObjectMapper will take the last value associated with this duplicate key. // For example: for {"key": 1, "key":2}, we will get "key"->2. val mapper = new ObjectMapper() - iter.map(record => mapper.readValue(record, classOf[Object])) - }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) + iter.map(record => mapper.readValue(record, classOf[java.util.Map[String, Any]])) + }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) } private def toLong(value: Any): Long = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index f580818e899b5..020baf0c7ec6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.api.java - import scala.beans.BeanProperty import org.scalatest.FunSuite From 1f908ce37100a2b925cd5e272ad8afde92bc614d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 16:07:21 -0700 Subject: [PATCH 21/25] Remove extra line. --- .../main/java/org/apache/spark/examples/sql/JavaSparkSQL.java | 1 - 1 file changed, 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 4a75f965cd170..607df3eddd550 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -57,7 +57,6 @@ public static void main(String[] args) throws Exception { JavaSparkContext ctx = new JavaSparkContext(sparkConf); JavaSQLContext sqlCtx = new JavaSQLContext(ctx); - System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( From 5428451a27ad8341c2b62529ac8efada000a14fd Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 16:23:49 -0700 Subject: [PATCH 22/25] Newline --- .../scala/org/apache/spark/sql/catalyst/plans/package.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala index 615d5739c8f3b..42bdab42b79ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala @@ -21,4 +21,4 @@ package org.apache.spark.sql.catalyst * A a collection of common abstractions for query plans as well as * a base logical plan representation. */ -package object plans \ No newline at end of file +package object plans From 79ea9baa67e9ae24d39503f16e29731d40dccfd2 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 16 Jun 2014 19:32:57 -0700 Subject: [PATCH 23/25] Fix typos. --- docs/sql-programming-guide.md | 30 +++++++++---------- python/pyspark/sql.py | 4 +-- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/api/java/JavaSQLContext.scala | 2 +- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index cd953a514148f..7b00016f289d5 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -64,7 +64,7 @@ descendants. To create a basic SQLContext, all you need is a SparkContext. val sc: SparkContext // An existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. import sqlContext.createSchemaRDD {% endhighlight %} @@ -102,17 +102,17 @@ sqlContext = SQLContext(sc)
-Spark SQL supports operating of a variety of data sources though the SchemaRDD interface. +Spark SQL supports operating on a variety of data sources though the SchemaRDD interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-Spark SQL supports operating of a variety of data sources though the JavaSchemaRDD interface. +Spark SQL supports operating on a variety of data sources though the JavaSchemaRDD interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-Spark SQL supports operating of a variety of data sources though the SchemaRDD interface. +Spark SQL supports operating on a variety of data sources though the SchemaRDD interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
@@ -132,7 +132,7 @@ registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} // sc is an existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. import sqlContext.createSchemaRDD // Define the schema using a case class. @@ -279,7 +279,7 @@ of the original data. Using the data from the above example: {% highlight scala %} // sqlContext from the previous example is used in this example. -// createSchemaRDD is used to implicitly convert a RDD to a SchemaRDD. +// createSchemaRDD is used to implicitly convert an RDD to a SchemaRDD. import sqlContext.createSchemaRDD val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. @@ -365,10 +365,10 @@ This conversion can be done using one of two methods in a SQLContext: // sc is an existing SparkContext. val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// A JSON dataset is pointed by path. +// A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. val path = "examples/src/main/resources/people.json" -// Create a SchemaRDD from the file(s) pointed by path +// Create a SchemaRDD from the file(s) pointed to by path val people = sqlContext.jsonFile(path) // The inferred schema can be visualized using the printSchema() method. @@ -385,7 +385,7 @@ people.registerAsTable("people") val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") // Alternatively, a SchemaRDD can be created for a JSON dataset represented by -// a RDD[String] storing one JSON object per string. +// an RDD[String] storing one JSON object per string. val anotherPeopleRDD = sc.parallelize( """{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil) val anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) @@ -404,10 +404,10 @@ This conversion can be done using one of two methods in a JavaSQLContext : // sc is an existing JavaSparkContext. JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); -// A JSON dataset is pointed by path. +// A JSON dataset is pointed to by path. // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; -// Create a JavaSchemaRDD from the file(s) pointed by path +// Create a JavaSchemaRDD from the file(s) pointed to by path JavaSchemaRDD people = sqlContext.jsonFile(path); // The inferred schema can be visualized using the printSchema() method. @@ -424,7 +424,7 @@ people.registerAsTable("people"); JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // Alternatively, a JavaSchemaRDD can be created for a JSON dataset represented by -// a RDD[String] storing one JSON object per string. +// an RDD[String] storing one JSON object per string. List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = sc.parallelize(jsonData); @@ -444,10 +444,10 @@ This conversion can be done using one of two methods in a SQLContext: from pyspark.sql import SQLContext sqlContext = SQLContext(sc) -# A JSON dataset is pointed by path. +# A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. path = "examples/src/main/resources/people.json" -# Create a SchemaRDD from the file(s) pointed by path +# Create a SchemaRDD from the file(s) pointed to by path people = sqlContext.jsonFile(path) # The inferred schema can be visualized using the printSchema() method. @@ -464,7 +464,7 @@ people.registerAsTable("people") teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") # Alternatively, a SchemaRDD can be created for a JSON dataset represented by -# a RDD[String] storing one JSON object per string. +# an RDD[String] storing one JSON object per string. anotherPeopleRDD = sc.parallelize([ '{"name":"Yin","address":{"city":"Columbus","state":"Ohio"}}']) anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index e511f63769cf0..966cac141eb33 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -161,7 +161,7 @@ def jsonFile(self, path): return SchemaRDD(jschema_rdd, self) def jsonRDD(self, rdd): - """Loads a RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. + """Loads an RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. >>> srdd = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd, "table1") @@ -310,7 +310,7 @@ class SchemaRDD(RDD): For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the L{SchemaRDD} is not operated on directly, as it's underlying - implementation is a RDD composed of Java objects. Instead it is + implementation is an RDD composed of Java objects. Instead it is converted to a PythonRDD in the JVM, on which Python operations can be done. """ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9d72c212538d4..2f5dd31dd0944 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -113,7 +113,7 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a * [[SchemaRDD]]. * * @group userf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 2108e51e5b8e0..2892f7eabc569 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -110,7 +110,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { jsonRDD(sqlContext.sparkContext.textFile(path)) /** - * Loads a RDD[String] storing JSON objects (one object per record), returning the result as a + * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a * [[JavaSchemaRDD]]. * * @group userf From 94ffdaaa024694bd91533077682976301303ea9d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 Jun 2014 12:09:37 -0700 Subject: [PATCH 24/25] Remove "get" from method names. --- python/pyspark/sql.py | 4 +-- .../org/apache/spark/sql/json/JsonRDD.scala | 32 +++++++++---------- .../org/apache/spark/sql/json/JsonSuite.scala | 18 +++++------ 3 files changed, 27 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 966cac141eb33..f3c18d17ac901 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -382,13 +382,13 @@ def saveAsTable(self, tableName): """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) - def getSchemaTreeString(self): + def schemaTreeString(self): """Returns the output schema in the tree format.""" return self._jschema_rdd.schemaTreeString() def printSchema(self): """Prints out the schema in the tree format.""" - print self.getSchemaTreeString() + print self.schemaTreeString() def count(self): """Return the number of elements in this RDD. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 5d24f8ecf2866..e7afe3bd1bce6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -38,7 +38,7 @@ private[sql] object JsonRDD extends Logging { require(samplingRatio > 0) val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) - val allKeys = parseJson(schemaData).map(getAllKeysWithValueTypes).reduce(_ ++ _) + val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) val baseSchema = createSchema(allKeys) @@ -64,7 +64,7 @@ private[sql] object JsonRDD extends Logging { 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)) + }.reduce((type1: DataType, type2: DataType) => compatibleType(type1, type2)) (fieldName, dataType) } @@ -111,7 +111,7 @@ private[sql] object JsonRDD extends Logging { /** * Returns the most general data type for two given data types. */ - private[json] def getCompatibleType(t1: DataType, t2: DataType): DataType = { + private[json] def compatibleType(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)) @@ -130,7 +130,7 @@ private[sql] object JsonRDD extends Logging { 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)) + (type1: DataType, type2: DataType) => compatibleType(type1, type2)) StructField(name, dataType, true) } } @@ -139,7 +139,7 @@ private[sql] object JsonRDD extends Logging { }) } case (ArrayType(elementType1), ArrayType(elementType2)) => - ArrayType(getCompatibleType(elementType1, elementType2)) + ArrayType(compatibleType(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 @@ -147,7 +147,7 @@ private[sql] object JsonRDD extends Logging { } } - private def getPrimitiveType(value: Any): DataType = { + private def typeOfPrimitiveValue(value: Any): DataType = { value match { case value: java.lang.String => StringType case value: java.lang.Integer => IntegerType @@ -166,11 +166,11 @@ private[sql] object JsonRDD extends Logging { /** * 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 + * to detect any possible type conflict. We use [[compatibleType]] 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 = { + private def typeOfArray(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. @@ -183,10 +183,10 @@ private[sql] object JsonRDD extends Logging { 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) + case seq: Seq[_] => typeOfArray(seq) + case value => typeOfPrimitiveValue(value) } - }.reduce((type1: DataType, type2: DataType) => getCompatibleType(type1, type2)) + }.reduce((type1: DataType, type2: DataType) => compatibleType(type1, type2)) ArrayType(elementType) } @@ -199,7 +199,7 @@ private[sql] object JsonRDD extends Logging { * 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)] = { + private def allKeysWithValueTypes(m: Map[String, Any]): Set[(String, DataType)] = { m.map{ // Quote the key with backticks to handle cases which have dots // in the field name. @@ -207,17 +207,17 @@ private[sql] object JsonRDD extends Logging { }.flatMap { case (key: String, struct: Map[String, Any]) => { // The value associted with the key is an JSON object. - getAllKeysWithValueTypes(struct).map { + allKeysWithValueTypes(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 { + typeOfArray(array) match { case ArrayType(StructType(Nil)) => { // The elements of this arrays are structs. array.asInstanceOf[List[Map[String, Any]]].flatMap { - element => getAllKeysWithValueTypes(element) + element => allKeysWithValueTypes(element) }.map { case (k, dataType) => (s"$key.$k", dataType) } :+ (key, ArrayType(StructType(Nil))) @@ -225,7 +225,7 @@ private[sql] object JsonRDD extends Logging { case ArrayType(elementType) => (key, ArrayType(elementType)) :: Nil } } - case (key: String, value) => (key, getPrimitiveType(value)) :: Nil + case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil }.toSet } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 79a9ba8fe7dcf..10bd9f08f0238 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, getCompatibleType} +import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ @@ -58,10 +58,10 @@ class JsonSuite extends QueryTest { test("Get compatible type") { def checkDataType(t1: DataType, t2: DataType, expected: DataType) { - var actual = getCompatibleType(t1, t2) + var actual = compatibleType(t1, t2) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") - actual = getCompatibleType(t2, t1) + actual = compatibleType(t2, t1) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") } @@ -494,12 +494,12 @@ class JsonSuite extends QueryTest { val expectedSchema = AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + AttributeReference("boolean", BooleanType, true)() :: + AttributeReference("double", DoubleType, true)() :: + AttributeReference("integer", IntegerType, true)() :: + AttributeReference("long", LongType, true)() :: + AttributeReference("null", StringType, true)() :: + AttributeReference("string", StringType, true)() :: Nil comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) From ce8eedd991e6bc45f282d4d147834d3b56ef6700 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 17 Jun 2014 17:32:00 -0700 Subject: [PATCH 25/25] rxin's comments. --- docs/sql-programming-guide.md | 9 +++------ project/SparkBuild.scala | 2 +- python/pyspark/sql.py | 8 +++++--- .../spark/sql/catalyst/plans/QueryPlan.scala | 16 ++++++++-------- sql/core/pom.xml | 4 ++-- .../scala/org/apache/spark/sql/SQLContext.scala | 2 ++ .../scala/org/apache/spark/sql/SchemaRDD.scala | 3 +++ .../org/apache/spark/sql/SchemaRDDLike.scala | 4 ++-- .../spark/sql/api/java/JavaSQLContext.scala | 2 ++ .../org/apache/spark/sql/json/JsonRDD.scala | 4 +--- 10 files changed, 29 insertions(+), 25 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 7b00016f289d5..522c83884ef42 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -102,17 +102,17 @@ sqlContext = SQLContext(sc)
-Spark SQL supports operating on a variety of data sources though the SchemaRDD interface. +Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-Spark SQL supports operating on a variety of data sources though the JavaSchemaRDD interface. +Spark SQL supports operating on a variety of data sources through the `JavaSchemaRDD` interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
-Spark SQL supports operating on a variety of data sources though the SchemaRDD interface. +Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
@@ -373,7 +373,6 @@ val people = sqlContext.jsonFile(path) // The inferred schema can be visualized using the printSchema() method. people.printSchema() -// The schema of people is ... // root // |-- age: IntegerType // |-- name: StringType @@ -412,7 +411,6 @@ JavaSchemaRDD people = sqlContext.jsonFile(path); // The inferred schema can be visualized using the printSchema() method. people.printSchema(); -// The schema of people is ... // root // |-- age: IntegerType // |-- name: StringType @@ -452,7 +450,6 @@ people = sqlContext.jsonFile(path) # The inferred schema can be visualized using the printSchema() method. people.printSchema() -# The schema of people is ... # root # |-- age: IntegerType # |-- name: StringType diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4c3b1218b2db4..7bb39dc77120b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -503,7 +503,7 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "com.twitter" % "parquet-column" % parquetVersion, "com.twitter" % "parquet-hadoop" % parquetVersion, - "com.fasterxml.jackson.core" % "jackson-core" % "2.3.2" + "com.fasterxml.jackson.core" % "jackson-databind" % "2.3.0" // json4s-jackson 3.2.6 requires jackson-databind 2.3.0. ), initialCommands in console := """ diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f3c18d17ac901..5051c82da32a7 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -141,6 +141,7 @@ def parquetFile(self, path): def jsonFile(self, path): """Loads a text file storing one JSON object per line, returning the result as a L{SchemaRDD}. + It goes through the entire dataset once to determine the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -162,6 +163,7 @@ def jsonFile(self, path): def jsonRDD(self, rdd): """Loads an RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. + It goes through the entire dataset once to determine the schema. >>> srdd = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd, "table1") @@ -382,13 +384,13 @@ def saveAsTable(self, tableName): """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) - def schemaTreeString(self): + def schemaString(self): """Returns the output schema in the tree format.""" - return self._jschema_rdd.schemaTreeString() + return self._jschema_rdd.schemaString() def printSchema(self): """Prints out the schema in the tree format.""" - print self.schemaTreeString() + print self.schemaString() def count(self): """Return the number of elements in this RDD. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 8452eb98a3be7..00e2d3bc24be9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -126,7 +126,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - protected def generateSchemaTreeString(schema: Seq[Attribute]): String = { + protected def generateSchemaString(schema: Seq[Attribute]): String = { val builder = new StringBuilder builder.append("root\n") val prefix = " |" @@ -136,10 +136,10 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy dataType match { case fields: StructType => builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaTreeString(fields, s"$prefix |", builder) + generateSchemaString(fields, s"$prefix |", builder) case ArrayType(fields: StructType) => builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaTreeString(fields, s"$prefix |", builder) + generateSchemaString(fields, s"$prefix |", builder) case ArrayType(elementType: DataType) => builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") case _ => builder.append(s"$prefix-- $name: $dataType\n") @@ -149,17 +149,17 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy builder.toString() } - protected def generateSchemaTreeString( + protected def generateSchemaString( schema: StructType, prefix: String, builder: StringBuilder): StringBuilder = { schema.fields.foreach { case StructField(name, fields: StructType, _) => builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaTreeString(fields, s"$prefix |", builder) + generateSchemaString(fields, s"$prefix |", builder) case StructField(name, ArrayType(fields: StructType), _) => builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaTreeString(fields, s"$prefix |", builder) + generateSchemaString(fields, s"$prefix |", builder) case StructField(name, ArrayType(elementType: DataType), _) => builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") case StructField(name, fieldType: DataType, _) => @@ -170,8 +170,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy } /** Returns the output schema in the tree format. */ - def schemaTreeString: String = generateSchemaTreeString(output) + def schemaString: String = generateSchemaString(output) /** Prints out the schema in the tree format */ - def printSchema(): Unit = println(schemaTreeString) + def printSchema(): Unit = println(schemaString) } diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f655f23e7961a..8210fd1f210d1 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -62,8 +62,8 @@ com.fasterxml.jackson.core - jackson-core - 2.3.2 + jackson-databind + 2.3.0 org.scalatest diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 2f5dd31dd0944..f7e03323bed33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -98,6 +98,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. + * It goes through the entire dataset once to determine the schema. * * @group userf */ @@ -115,6 +116,7 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a * [[SchemaRDD]]. + * It goes through the entire dataset once to determine the schema. * * @group userf */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index bcbeee31681b3..7c0efb4566610 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -343,6 +343,9 @@ class SchemaRDD( */ def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) + /** + * Converts a JavaRDD to a PythonRDD. It is used by pyspark. + */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { def rowToMap(row: Row, structType: StructType): JMap[String, Any] = { val fields = structType.fields.map(field => (field.name, field.dataType)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 2b722547dbd0e..fe81721943202 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -124,8 +124,8 @@ private[sql] trait SchemaRDDLike { sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd /** Returns the output schema in the tree format. */ - def schemaTreeString: String = queryExecution.analyzed.schemaTreeString + def schemaString: String = queryExecution.analyzed.schemaString /** Prints out the schema in the tree format. */ - def printSchema(): Unit = println(schemaTreeString) + def printSchema(): Unit = println(schemaString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 2892f7eabc569..ff9842267ffe0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -103,6 +103,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { /** * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. + * It goes through the entire dataset once to determine the schema. * * @group userf */ @@ -112,6 +113,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { /** * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a * [[JavaSchemaRDD]]. + * It goes through the entire dataset once to determine the schema. * * @group userf */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index e7afe3bd1bce6..edf86775579d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -35,11 +35,9 @@ private[sql] object JsonRDD extends Logging { private[sql] def inferSchema( json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { - require(samplingRatio > 0) + require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) - val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) - val baseSchema = createSchema(allKeys) createLogicalPlan(json, baseSchema)