Skip to content

Commit

Permalink
address comments
Browse files Browse the repository at this point in the history
  • Loading branch information
Davies Liu committed Feb 13, 2016
1 parent 016c36c commit a4bae33
Show file tree
Hide file tree
Showing 3 changed files with 49 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@ case class ScalarSubquery(query: LogicalPlan) extends SubqueryExpression with Co

override def withNewPlan(plan: LogicalPlan): ScalarSubquery = ScalarSubquery(plan)

// TODO: support sql()

// the first column in first row from `query`.
private var result: Any = null

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.types.BooleanType
import org.apache.spark.unsafe.types.CalendarInterval

class CatalystQlSuite extends PlanTest {
Expand Down Expand Up @@ -203,10 +204,47 @@ class CatalystQlSuite extends PlanTest {
}

test("subquery") {
parser.parsePlan("select (select max(b) from s) ss from t")
parser.parsePlan("select * from t where a = (select b from s)")
parser.parsePlan("select * from t where a > (select b from s)")
parser.parsePlan("select * from t group by g having a = (select b from s)")
parser.parsePlan("select * from t group by g having a > (select b from s)")
comparePlans(
parser.parsePlan("select (select max(b) from s) ss from t"),
Project(
UnresolvedAlias(
Alias(
ScalarSubquery(
Project(
UnresolvedAlias(
UnresolvedFunction("max", UnresolvedAttribute("b") :: Nil, false)) :: Nil,
UnresolvedRelation(TableIdentifier("s")))),
"ss")(ExprId(0))) :: Nil,
UnresolvedRelation(TableIdentifier("t"))))
comparePlans(
parser.parsePlan("select * from t where a = (select b from s)"),
Project(
UnresolvedAlias(
UnresolvedStar(None)) :: Nil,
Filter(
EqualTo(
UnresolvedAttribute("a"),
ScalarSubquery(
Project(
UnresolvedAlias(
UnresolvedAttribute("b")) :: Nil,
UnresolvedRelation(TableIdentifier("s"))))),
UnresolvedRelation(TableIdentifier("t")))))
comparePlans(
parser.parsePlan("select * from t group by g having a > (select b from s)"),
Filter(
Cast(
GreaterThan(
UnresolvedAttribute("a"),
ScalarSubquery(
Project(
UnresolvedAlias(
UnresolvedAttribute("b")) :: Nil,
UnresolvedRelation(TableIdentifier("s"))))),
BooleanType),
Aggregate(
UnresolvedAttribute("g") :: Nil,
UnresolvedAlias(UnresolvedStar(None)) :: Nil,
UnresolvedRelation(TableIdentifier("t")))))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2115,6 +2115,10 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
"select a from (select 1 as a union all select 2 as a) t " +
"where a = (select max(b) from t2) ").collect()
}

assertResult(Array(Row(3))) {
sql("select (select (select 1) + 1) + 1").collect()
}
}

test("SPARK-13056: Null in map value causes NPE") {
Expand Down

0 comments on commit a4bae33

Please sign in to comment.