From 367d237b3d5e445a67e6a8b9c9ae79abff26a045 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 7 Oct 2014 13:17:02 -0700 Subject: [PATCH] add test --- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 1 + .../sql/catalyst/expressions/generators.scala | 2 +- .../expressions/namedExpressions.scala | 3 ++- .../plans/logical/basicOperators.scala | 2 +- .../spark/sql/catalyst/types/dataTypes.scala | 3 ++- .../org/apache/spark/sql/MetadataSuite.scala | 27 +++++++++++++++++++ 7 files changed, 35 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/MetadataSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 88a8fa7c28e0f..51ba0c3dac321 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -62,7 +62,7 @@ object ScalaReflection { params.head.map { p => val Schema(dataType, nullable) = schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) - StructField(p.name.toString, dataType, nullable) + StructField(p.name.toString, dataType, nullable, Map.empty) }), nullable = true) // Need to decide if we actually need a special type here. case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 70507e7ee2be8..ab0179b14b592 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -41,6 +41,7 @@ abstract class Expression extends TreeNode[Expression] { */ def foldable: Boolean = false def nullable: Boolean + def metadata: Map[String, Any] = Map.empty def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator)) /** Returns the result of evaluating this expression on a given input Row */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 9c865254e0be9..ab0701fd9a80b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -43,7 +43,7 @@ abstract class Generator extends Expression { override type EvaluatedType = TraversableOnce[Row] override lazy val dataType = - ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) + ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))) override def nullable = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 755b94c2a123b..5564fa2b09bb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -40,7 +40,6 @@ abstract class NamedExpression extends Expression { def name: String def exprId: ExprId def qualifiers: Seq[String] - def metadata: Map[String, Any] = Map.empty def toAttribute: Attribute @@ -99,6 +98,8 @@ case class Alias(child: Expression, name: String) override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" override protected final def otherCopyArgs = exprId :: qualifiers :: Nil + + override def metadata: Map[String, Any] = child.metadata } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 5d10754c7b028..6cd2b456ec7c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -163,7 +163,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { protected def lowerCaseSchema(dataType: DataType): DataType = dataType match { case StructType(fields) => StructType(fields.map(f => - StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) + StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable, f.metadata))) case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) case otherType => otherType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 663e532bcff82..99bd12dfa5e19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -57,7 +57,8 @@ object DataType extends RegexParsers { protected lazy val structField: Parser[StructField] = ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { case name ~ tpe ~ nullable => - StructField(name, tpe, nullable = nullable) + // TODO: parse metadata + StructField(name, tpe, nullable = nullable, Map.empty) } protected lazy val boolVal: Parser[Boolean] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MetadataSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MetadataSuite.scala new file mode 100644 index 0000000000000..3512998f9d832 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/MetadataSuite.scala @@ -0,0 +1,27 @@ +package org.apache.spark.sql + +import org.apache.spark.sql.test.TestSQLContext +import org.scalatest.FunSuite + +case class Person(name: String, age: Int) + +class MetadataSuite extends FunSuite { + + test("metadata") { + val sqlContext = TestSQLContext + import sqlContext._ + val members = sqlContext.sparkContext.makeRDD(Seq( + Person("mike", 10), + Person("jim", 20))) + val table: SchemaRDD = sqlContext.createSchemaRDD(members) + val schema: StructType = table.schema + println("schema: " + schema) + val ageField = schema("age").copy(metadata = Map("desc" -> "age (must be nonnegative)")) + val newSchema = schema.copy(Seq(schema("name"), ageField)) + val newTable = sqlContext.applySchema(table, newSchema) + val selectByExprAgeField = newTable.select('age).schema("age") + assert(selectByExprAgeField.metadata.nonEmpty) + val selectByNameAttrAgeField = newTable.select("age".attr).schema("age") + assert(selectByNameAttrAgeField.metadata.nonEmpty) + } +}