Skip to content

Commit d65072e

Browse files
committed
remove Map.empty
1 parent 367d237 commit d65072e

File tree

6 files changed

+30
-20
lines changed

6 files changed

+30
-20
lines changed

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ object ScalaReflection {
4343
/** Returns a Sequence of attributes for the given case class type. */
4444
def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match {
4545
case Schema(s: StructType, _) =>
46-
s.fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)())
46+
s.fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())
4747
}
4848

4949
/** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */
@@ -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, Map.empty)
65+
StructField(p.name.toString, dataType, nullable)
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: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,9 +41,11 @@ abstract class Expression extends TreeNode[Expression] {
4141
*/
4242
def foldable: Boolean = false
4343
def nullable: Boolean
44-
def metadata: Map[String, Any] = Map.empty
4544
def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator))
4645

46+
/** Returns the metadata when an expression is a reference to another expression with metadata. */
47+
def metadata: Map[String, Any] = Map.empty
48+
4749
/** Returns the result of evaluating this expression on a given input Row */
4850
def eval(input: Row = null): EvaluatedType
4951

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

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -85,11 +85,11 @@ case class Alias(child: Expression, name: String)
8585

8686
override def dataType = child.dataType
8787
override def nullable = child.nullable
88-
88+
override def metadata: Map[String, Any] = child.metadata
8989

9090
override def toAttribute = {
9191
if (resolved) {
92-
AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers)
92+
AttributeReference(name, child.dataType, child.nullable, child.metadata)(exprId, qualifiers)
9393
} else {
9494
UnresolvedAttribute(name)
9595
}
@@ -98,8 +98,6 @@ case class Alias(child: Expression, name: String)
9898
override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix"
9999

100100
override protected final def otherCopyArgs = exprId :: qualifiers :: Nil
101-
102-
override def metadata: Map[String, Any] = child.metadata
103101
}
104102

105103
/**
@@ -108,6 +106,7 @@ case class Alias(child: Expression, name: String)
108106
* @param name The name of this attribute, should only be used during analysis or for debugging.
109107
* @param dataType The [[DataType]] of this attribute.
110108
* @param nullable True if null is a valid value for this attribute.
109+
* @param metadata The metadata of this attribute.
111110
* @param exprId A globally unique id used to check if different AttributeReferences refer to the
112111
* same attribute.
113112
* @param qualifiers a list of strings that can be used to referred to this attribute in a fully

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

Lines changed: 5 additions & 4 deletions
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, f.metadata)))
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
}
@@ -173,9 +173,10 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode {
173173
AttributeReference(
174174
a.name.toLowerCase,
175175
lowerCaseSchema(a.dataType),
176-
a.nullable)(
177-
a.exprId,
178-
a.qualifiers)
176+
a.nullable,
177+
a.metadata)(
178+
a.exprId,
179+
a.qualifiers)
179180
case other => other
180181
}
181182
}

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

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -298,7 +298,8 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT
298298
* @param dataType The data type of this field.
299299
* @param nullable Indicates if values of this field can be `null` values.
300300
* @param metadata The metadata of this field, which is a map from string to simple type that can be
301-
* serialized to JSON automatically.
301+
* serialized to JSON automatically. The metadata should be preserved during
302+
* transformation if the content of the column is not modified, e.g, in selection.
302303
*/
303304
case class StructField(
304305
name: String,
@@ -330,8 +331,8 @@ case class StructType(fields: Seq[StructField]) extends DataType {
330331
* have a name matching the given name, `null` will be returned.
331332
*/
332333
def apply(name: String): StructField = {
333-
nameToField.get(name).getOrElse(
334-
throw new IllegalArgumentException(s"Field ${name} does not exist."))
334+
nameToField.getOrElse(name,
335+
throw new IllegalArgumentException(s"Field $name does not exist."))
335336
}
336337

337338
/**

sql/core/src/test/scala/org/apache/spark/sql/MetadataSuite.scala

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -13,15 +13,22 @@ class MetadataSuite extends FunSuite {
1313
val members = sqlContext.sparkContext.makeRDD(Seq(
1414
Person("mike", 10),
1515
Person("jim", 20)))
16-
val table: SchemaRDD = sqlContext.createSchemaRDD(members)
17-
val schema: StructType = table.schema
16+
val person: SchemaRDD = sqlContext.createSchemaRDD(members)
17+
val schema: StructType = person.schema
1818
println("schema: " + schema)
19-
val ageField = schema("age").copy(metadata = Map("desc" -> "age (must be nonnegative)"))
19+
val ageField = schema("age").copy(metadata = Map("doc" -> "age (must be nonnegative)"))
2020
val newSchema = schema.copy(Seq(schema("name"), ageField))
21-
val newTable = sqlContext.applySchema(table, newSchema)
21+
val newTable = sqlContext.applySchema(person, newSchema)
22+
newTable.registerTempTable("person")
2223
val selectByExprAgeField = newTable.select('age).schema("age")
23-
assert(selectByExprAgeField.metadata.nonEmpty)
24+
assert(selectByExprAgeField.metadata.contains("doc"))
2425
val selectByNameAttrAgeField = newTable.select("age".attr).schema("age")
25-
assert(selectByNameAttrAgeField.metadata.nonEmpty)
26+
assert(selectByNameAttrAgeField.metadata.contains("doc"))
27+
val selectAgeBySQL = sql("SELECT age FROM person").schema("age")
28+
println(selectAgeBySQL)
29+
assert(selectAgeBySQL.metadata.contains("doc"))
30+
val selectStarBySQL = sql("SELECT * FROM person").schema("age")
31+
println(selectStarBySQL)
32+
assert(selectStarBySQL.metadata.contains("doc"))
2633
}
2734
}

0 commit comments

Comments
 (0)