Skip to content

Commit

Permalink
address comments
Browse files Browse the repository at this point in the history
  • Loading branch information
cloud-fan committed May 8, 2015
1 parent 3f880c3 commit 6bf72bc
Show file tree
Hide file tree
Showing 2 changed files with 53 additions and 46 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -200,5 +200,5 @@ case class UnresolvedGetField(child: Expression, fieldExpr: Expression) extends
override def eval(input: Row = null): EvaluatedType =
throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")

override def toString: String = s"$child.getField($fieldExpr)"
override def toString: String = s"$child[$fieldExpr]"
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,13 @@ object GetField {
/**
* Returns the resolved `GetField`. It will return one kind of concrete `GetField`,
* depend on the type of `child` and `fieldExpr`.
*
* `child` | `fieldExpr` | concrete `GetField`
* -------------------------------------------------------------
* Struct | Literal String | SimpleStructGetField
* Array[Struct] | Literal String | ArrayStructGetField
* Array | Integral type | ArrayOrdinalGetField
* Map | Any type | MapOrdinalGetField
*/
def apply(
child: Expression,
Expand All @@ -45,22 +52,27 @@ object GetField {
case (_: MapType, _) =>
MapOrdinalGetField(child, fieldExpr)
case (otherType, _) =>
throw new AnalysisException(
"GetField is not valid on child of type " +
s"$otherType with fieldExpr of type ${fieldExpr.dataType}")
val errorMsg = otherType match {
case StructType(_) | ArrayType(StructType(_), _) =>
s"Field name should be String Literal, but it's $fieldExpr"
case _: ArrayType =>
s"Array index should be integral type, but it's ${fieldExpr.dataType}"
case other =>
s"Can't get field on $child"
}
throw new AnalysisException(errorMsg)
}
}

def unapply(g: GetField): Option[(Expression, Expression)] = {
g match {
case _: StructGetField => Some((g.child, null))
case o: OrdinalGetField => Some((o.child, o.ordinal))
case _ => None
case _ => Some((g.child, null))
}
}

/**
* find the ordinal of StructField, report error if no desired field or over one
* Find the ordinal of StructField, report error if no desired field or over one
* desired fields are found.
*/
private def findField(fields: Array[StructField], fieldName: String, resolver: Resolver): Int = {
Expand All @@ -84,51 +96,16 @@ trait GetField extends UnaryExpression {
type EvaluatedType = Any
}

abstract class StructGetField extends GetField {
self: Product =>

def field: StructField

override def foldable: Boolean = child.foldable
override def toString: String = s"$child.${field.name}"
}

abstract class OrdinalGetField extends GetField {
self: Product =>

def ordinal: Expression

/** `Null` is returned for invalid ordinals. */
override def nullable: Boolean = true
override def foldable: Boolean = child.foldable && ordinal.foldable
override def toString: String = s"$child[$ordinal]"
override def children: Seq[Expression] = child :: ordinal :: Nil

override def eval(input: Row): Any = {
val value = child.eval(input)
if (value == null) {
null
} else {
val o = ordinal.eval(input)
if (o == null) {
null
} else {
evalNotNull(value, o)
}
}
}

protected def evalNotNull(value: Any, ordinal: Any): Any
}

/**
* Returns the value of fields in the Struct `child`.
*/
case class SimpleStructGetField(child: Expression, field: StructField, ordinal: Int)
extends StructGetField {
extends GetField {

override def dataType: DataType = field.dataType
override def nullable: Boolean = child.nullable || field.nullable
override def foldable: Boolean = child.foldable
override def toString: String = s"$child.${field.name}"

override def eval(input: Row): Any = {
val baseValue = child.eval(input).asInstanceOf[Row]
Expand All @@ -143,10 +120,12 @@ case class ArrayStructGetField(
child: Expression,
field: StructField,
ordinal: Int,
containsNull: Boolean) extends StructGetField {
containsNull: Boolean) extends GetField {

override def dataType: DataType = ArrayType(field.dataType, containsNull)
override def nullable: Boolean = child.nullable
override def foldable: Boolean = child.foldable
override def toString: String = s"$child.${field.name}"

override def eval(input: Row): Any = {
val baseValue = child.eval(input).asInstanceOf[Seq[Row]]
Expand All @@ -158,6 +137,34 @@ case class ArrayStructGetField(
}
}

abstract class OrdinalGetField extends GetField {
self: Product =>

def ordinal: Expression

/** `Null` is returned for invalid ordinals. */
override def nullable: Boolean = true
override def foldable: Boolean = child.foldable && ordinal.foldable
override def toString: String = s"$child[$ordinal]"
override def children: Seq[Expression] = child :: ordinal :: Nil

override def eval(input: Row): Any = {
val value = child.eval(input)
if (value == null) {
null
} else {
val o = ordinal.eval(input)
if (o == null) {
null
} else {
evalNotNull(value, o)
}
}
}

protected def evalNotNull(value: Any, ordinal: Any): Any
}

/**
* Returns the field at `ordinal` in the Array `child`
*/
Expand Down

0 comments on commit 6bf72bc

Please sign in to comment.