Skip to content

Commit

Permalink
Fix case insensitive resolution of GetField.
Browse files Browse the repository at this point in the history
  • Loading branch information
marmbrus committed Dec 17, 2014
1 parent 3d0c37b commit 0a47aae
Show file tree
Hide file tree
Showing 3 changed files with 28 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
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.StructType

/**
* A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing
Expand Down Expand Up @@ -187,6 +188,15 @@ class Analyzer(catalog: Catalog,
val result = q.resolveChildren(name, resolver).getOrElse(u)
logDebug(s"Resolving $u to $result")
result

// Resolve field names using the resolver.
case f @ GetField(child, fieldName) if !f.resolved && child.resolved =>
child.dataType match {
case StructType(fields) =>
val resolvedFieldName = fields.map(_.name).find(resolver(_, fieldName))
resolvedFieldName.map(n => f.copy(fieldName = n)).getOrElse(f)
case _ => f
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,13 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio

lazy val ordinal = structType.fields.indexOf(field)

override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType]
override lazy val resolved = childrenResolved && fieldResolved

/** Returns true only if the fieldName is found in the child struct. */
private def fieldResolved = child.dataType match {
case StructType(fields) => fields.map(_.name).contains(fieldName)
case _ => false
}

override def eval(input: Row): Any = {
val baseValue = child.eval(input).asInstanceOf[Row]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,17 @@ case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested])
* A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
*/
class HiveResolutionSuite extends HiveComparisonTest {

case class NestedData(a: Seq[NestedData2], B: NestedData2)
case class NestedData2(a: NestedData3, B: NestedData3)
case class NestedData3(a: Int, B: Int)

test("SPARK-3698: case insensitive test for nested data") {
sparkContext.makeRDD(Seq.empty[NestedData]).registerTempTable("nested")
// This should be successfully analyzed
sql("SELECT a[0].A.A from nested").queryExecution.analyzed
}

createQueryTest("table.attr",
"SELECT src.key FROM src ORDER BY key LIMIT 1")

Expand Down

0 comments on commit 0a47aae

Please sign in to comment.