Skip to content

Commit 367d237

Browse files
committed
add test
1 parent c194d5e commit 367d237

File tree

7 files changed

+35
-5
lines changed

7 files changed

+35
-5
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ object ScalaReflection {
6262
params.head.map { p =>
6363
val Schema(dataType, nullable) =
6464
schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs))
65-
StructField(p.name.toString, dataType, nullable)
65+
StructField(p.name.toString, dataType, nullable, Map.empty)
6666
}), nullable = true)
6767
// Need to decide if we actually need a special type here.
6868
case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ abstract class Expression extends TreeNode[Expression] {
4141
*/
4242
def foldable: Boolean = false
4343
def nullable: Boolean
44+
def metadata: Map[String, Any] = Map.empty
4445
def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator))
4546

4647
/** Returns the result of evaluating this expression on a given input Row */

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ abstract class Generator extends Expression {
4343
override type EvaluatedType = TraversableOnce[Row]
4444

4545
override lazy val dataType =
46-
ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable))))
46+
ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))))
4747

4848
override def nullable = false
4949

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,6 @@ abstract class NamedExpression extends Expression {
4040
def name: String
4141
def exprId: ExprId
4242
def qualifiers: Seq[String]
43-
def metadata: Map[String, Any] = Map.empty
4443

4544
def toAttribute: Attribute
4645

@@ -99,6 +98,8 @@ case class Alias(child: Expression, name: String)
9998
override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix"
10099

101100
override protected final def otherCopyArgs = exprId :: qualifiers :: Nil
101+
102+
override def metadata: Map[String, Any] = child.metadata
102103
}
103104

104105
/**

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -163,7 +163,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode {
163163
protected def lowerCaseSchema(dataType: DataType): DataType = dataType match {
164164
case StructType(fields) =>
165165
StructType(fields.map(f =>
166-
StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable)))
166+
StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable, f.metadata)))
167167
case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull)
168168
case otherType => otherType
169169
}

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -57,7 +57,8 @@ object DataType extends RegexParsers {
5757
protected lazy val structField: Parser[StructField] =
5858
("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ {
5959
case name ~ tpe ~ nullable =>
60-
StructField(name, tpe, nullable = nullable)
60+
// TODO: parse metadata
61+
StructField(name, tpe, nullable = nullable, Map.empty)
6162
}
6263

6364
protected lazy val boolVal: Parser[Boolean] =
Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,27 @@
1+
package org.apache.spark.sql
2+
3+
import org.apache.spark.sql.test.TestSQLContext
4+
import org.scalatest.FunSuite
5+
6+
case class Person(name: String, age: Int)
7+
8+
class MetadataSuite extends FunSuite {
9+
10+
test("metadata") {
11+
val sqlContext = TestSQLContext
12+
import sqlContext._
13+
val members = sqlContext.sparkContext.makeRDD(Seq(
14+
Person("mike", 10),
15+
Person("jim", 20)))
16+
val table: SchemaRDD = sqlContext.createSchemaRDD(members)
17+
val schema: StructType = table.schema
18+
println("schema: " + schema)
19+
val ageField = schema("age").copy(metadata = Map("desc" -> "age (must be nonnegative)"))
20+
val newSchema = schema.copy(Seq(schema("name"), ageField))
21+
val newTable = sqlContext.applySchema(table, newSchema)
22+
val selectByExprAgeField = newTable.select('age).schema("age")
23+
assert(selectByExprAgeField.metadata.nonEmpty)
24+
val selectByNameAttrAgeField = newTable.select("age".attr).schema("age")
25+
assert(selectByNameAttrAgeField.metadata.nonEmpty)
26+
}
27+
}

0 commit comments

Comments
 (0)