Skip to content

Commit d956cc2

Browse files
marmbrusrxin
authored andcommitted
[SQL] Minor fixes.
Author: Michael Armbrust <michael@databricks.com> Closes #315 from marmbrus/minorFixes and squashes the following commits: b23a15d [Michael Armbrust] fix scaladoc 11062ac [Michael Armbrust] Fix registering "SELECT *" queries as tables and caching them. As some tests for this and self-joins. 3997dc9 [Michael Armbrust] Move Row extractor to catalyst. 208bf5e [Michael Armbrust] More idiomatic naming of DSL functions. * subquery => as * for join condition => on, i.e., `r.join(s, condition = 'a == 'b)` =>`r.join(s, on = 'a == 'b)` 87211ce [Michael Armbrust] Correctly handle self joins of in-memory cached tables. 69e195e [Michael Armbrust] Change != to !== in the DSL since != will always translate to != on Any. 01f2dd5 [Michael Armbrust] Correctly assign aliases to tables in SqlParser.
1 parent 198892f commit d956cc2

File tree

10 files changed

+53
-34
lines changed

10 files changed

+53
-34
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -219,7 +219,7 @@ class SqlParser extends StandardTokenParsers {
219219

220220
protected lazy val relationFactor: Parser[LogicalPlan] =
221221
ident ~ (opt(AS) ~> opt(ident)) ^^ {
222-
case ident ~ alias => UnresolvedRelation(alias, ident)
222+
case tableName ~ alias => UnresolvedRelation(None, tableName, alias)
223223
} |
224224
"(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) }
225225

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ package object dsl {
7070
def > (other: Expression) = GreaterThan(expr, other)
7171
def >= (other: Expression) = GreaterThanOrEqual(expr, other)
7272
def === (other: Expression) = Equals(expr, other)
73-
def != (other: Expression) = Not(Equals(expr, other))
73+
def !== (other: Expression) = Not(Equals(expr, other))
7474

7575
def like(other: Expression) = Like(expr, other)
7676
def rlike(other: Expression) = RLike(expr, other)

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

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,21 @@ package org.apache.spark.sql.catalyst.expressions
1919

2020
import org.apache.spark.sql.catalyst.types.NativeType
2121

22+
object Row {
23+
/**
24+
* This method can be used to extract fields from a [[Row]] object in a pattern match. Example:
25+
* {{{
26+
* import org.apache.spark.sql._
27+
*
28+
* val pairs = sql("SELECT key, value FROM src").rdd.map {
29+
* case Row(key: Int, value: String) =>
30+
* key -> value
31+
* }
32+
* }}}
33+
*/
34+
def unapplySeq(row: Row): Some[Seq[Any]] = Some(row)
35+
}
36+
2237
/**
2338
* Represents one row of output from a relational operator. Allows both generic access by ordinal,
2439
* which will incur boxing overhead for primitives, as well as native primitive access.

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -162,6 +162,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode {
162162
a.nullable)(
163163
a.exprId,
164164
a.qualifiers)
165+
case other => other
165166
}
166167

167168
def references = Set.empty

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

Lines changed: 1 addition & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -32,18 +32,5 @@ package object sql {
3232

3333
type Row = catalyst.expressions.Row
3434

35-
object Row {
36-
/**
37-
* This method can be used to extract fields from a [[Row]] object in a pattern match. Example:
38-
* {{{
39-
* import org.apache.spark.sql._
40-
*
41-
* val pairs = sql("SELECT key, value FROM src").rdd.map {
42-
* case Row(key: Int, value: String) =>
43-
* key -> value
44-
* }
45-
* }}}
46-
*/
47-
def unapplySeq(row: Row): Some[Seq[Any]] = Some(row)
48-
}
35+
val Row = catalyst.expressions.Row
4936
}

sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -148,17 +148,17 @@ class SchemaRDD(
148148
*
149149
* @param otherPlan the [[SchemaRDD]] that should be joined with this one.
150150
* @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.`
151-
* @param condition An optional condition for the join operation. This is equivilent to the `ON`
152-
* clause in standard SQL. In the case of `Inner` joins, specifying a
153-
* `condition` is equivilent to adding `where` clauses after the `join`.
151+
* @param on An optional condition for the join operation. This is equivilent to the `ON`
152+
* clause in standard SQL. In the case of `Inner` joins, specifying a
153+
* `condition` is equivilent to adding `where` clauses after the `join`.
154154
*
155155
* @group Query
156156
*/
157157
def join(
158158
otherPlan: SchemaRDD,
159159
joinType: JoinType = Inner,
160-
condition: Option[Expression] = None): SchemaRDD =
161-
new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, condition))
160+
on: Option[Expression] = None): SchemaRDD =
161+
new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, on))
162162

163163
/**
164164
* Sorts the results by the given expressions.
@@ -195,14 +195,14 @@ class SchemaRDD(
195195
* with the same name, for example, when peforming self-joins.
196196
*
197197
* {{{
198-
* val x = schemaRDD.where('a === 1).subquery('x)
199-
* val y = schemaRDD.where('a === 2).subquery('y)
198+
* val x = schemaRDD.where('a === 1).as('x)
199+
* val y = schemaRDD.where('a === 2).as('y)
200200
* x.join(y).where("x.a".attr === "y.a".attr),
201201
* }}}
202202
*
203203
* @group Query
204204
*/
205-
def subquery(alias: Symbol) =
205+
def as(alias: Symbol) =
206206
new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan))
207207

208208
/**

sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
2424
import org.apache.spark.sql.catalyst.expressions.GenericRow
2525
import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical}
2626
import org.apache.spark.sql.catalyst.plans.physical._
27+
import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
2728

2829
abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
2930
self: Product =>
@@ -69,6 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
6970
SparkLogicalPlan(
7071
alreadyPlanned match {
7172
case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd)
73+
case InMemoryColumnarTableScan(output, child) =>
74+
InMemoryColumnarTableScan(output.map(_.newInstance), child)
7275
case _ => sys.error("Multiple instance of the same relation detected.")
7376
}).asInstanceOf[this.type]
7477
}

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

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -58,4 +58,17 @@ class CachedTableSuite extends QueryTest {
5858
TestSQLContext.uncacheTable("testData")
5959
}
6060
}
61+
62+
test("SELECT Star Cached Table") {
63+
TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar")
64+
TestSQLContext.cacheTable("selectStar")
65+
TestSQLContext.sql("SELECT * FROM selectStar")
66+
TestSQLContext.uncacheTable("selectStar")
67+
}
68+
69+
test("Self-join cached") {
70+
TestSQLContext.cacheTable("testData")
71+
TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key")
72+
TestSQLContext.uncacheTable("testData")
73+
}
6174
}

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

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -119,8 +119,8 @@ class DslQuerySuite extends QueryTest {
119119
}
120120

121121
test("inner join, where, multiple matches") {
122-
val x = testData2.where('a === 1).subquery('x)
123-
val y = testData2.where('a === 1).subquery('y)
122+
val x = testData2.where('a === 1).as('x)
123+
val y = testData2.where('a === 1).as('y)
124124
checkAnswer(
125125
x.join(y).where("x.a".attr === "y.a".attr),
126126
(1,1,1,1) ::
@@ -131,17 +131,17 @@ class DslQuerySuite extends QueryTest {
131131
}
132132

133133
test("inner join, no matches") {
134-
val x = testData2.where('a === 1).subquery('x)
135-
val y = testData2.where('a === 2).subquery('y)
134+
val x = testData2.where('a === 1).as('x)
135+
val y = testData2.where('a === 2).as('y)
136136
checkAnswer(
137137
x.join(y).where("x.a".attr === "y.a".attr),
138138
Nil)
139139
}
140140

141141
test("big inner join, 4 matches per row") {
142142
val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData)
143-
val bigDataX = bigData.subquery('x)
144-
val bigDataY = bigData.subquery('y)
143+
val bigDataX = bigData.as('x)
144+
val bigDataY = bigData.as('y)
145145

146146
checkAnswer(
147147
bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr),
@@ -181,8 +181,8 @@ class DslQuerySuite extends QueryTest {
181181
}
182182

183183
test("full outer join") {
184-
val left = upperCaseData.where('N <= 4).subquery('left)
185-
val right = upperCaseData.where('N >= 3).subquery('right)
184+
val left = upperCaseData.where('N <= 4).as('left)
185+
val right = upperCaseData.where('N >= 3).as('right)
186186

187187
checkAnswer(
188188
left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)),

sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -56,8 +56,8 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll {
5656
}
5757

5858
test("self-join parquet files") {
59-
val x = ParquetTestData.testData.subquery('x)
60-
val y = ParquetTestData.testData.subquery('y)
59+
val x = ParquetTestData.testData.as('x)
60+
val y = ParquetTestData.testData.as('y)
6161
val query = x.join(y).where("x.myint".attr === "y.myint".attr)
6262

6363
// Check to make sure that the attributes from either side of the join have unique expression

0 commit comments

Comments
 (0)