Skip to content

Commit d0bd412

Browse files
committed
Merge remote-tracking branch 'upstream/master' into newJson
2 parents ab810b0 + ac96d96 commit d0bd412

16 files changed

Lines changed: 252 additions & 167 deletions

File tree

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

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.catalyst.plans.logical
1919

20-
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute}
20+
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference}
2121
import org.apache.spark.sql.catalyst.types.StringType
2222

2323
/**
@@ -26,35 +26,37 @@ import org.apache.spark.sql.catalyst.types.StringType
2626
*/
2727
abstract class Command extends LeafNode {
2828
self: Product =>
29-
def output: Seq[Attribute] = Seq.empty // TODO: SPARK-2081 should fix this
29+
def output: Seq[Attribute] = Seq.empty
3030
}
3131

3232
/**
3333
* Returned for commands supported by a given parser, but not catalyst. In general these are DDL
3434
* commands that are passed directly to another system.
3535
*/
36-
case class NativeCommand(cmd: String) extends Command
36+
case class NativeCommand(cmd: String) extends Command {
37+
override def output =
38+
Seq(BoundReference(0, AttributeReference("result", StringType, nullable = false)()))
39+
}
3740

3841
/**
3942
* Commands of the form "SET (key) (= value)".
4043
*/
4144
case class SetCommand(key: Option[String], value: Option[String]) extends Command {
4245
override def output = Seq(
43-
AttributeReference("key", StringType, nullable = false)(),
44-
AttributeReference("value", StringType, nullable = false)()
45-
)
46+
BoundReference(0, AttributeReference("key", StringType, nullable = false)()),
47+
BoundReference(1, AttributeReference("value", StringType, nullable = false)()))
4648
}
4749

4850
/**
4951
* Returned by a parser when the users only wants to see what query plan would be executed, without
5052
* actually performing the execution.
5153
*/
5254
case class ExplainCommand(plan: LogicalPlan) extends Command {
53-
override def output = Seq(AttributeReference("plan", StringType, nullable = false)())
55+
override def output =
56+
Seq(BoundReference(0, AttributeReference("plan", StringType, nullable = false)()))
5457
}
5558

5659
/**
5760
* Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command.
5861
*/
5962
case class CacheCommand(tableName: String, doCache: Boolean) extends Command
60-

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -161,7 +161,7 @@ class FilterPushdownSuite extends OptimizerTest {
161161

162162
comparePlans(optimized, correctAnswer)
163163
}
164-
164+
165165
test("joins: push down left outer join #1") {
166166
val x = testRelation.subquery('x)
167167
val y = testRelation.subquery('y)

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

Lines changed: 5 additions & 34 deletions
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.{ScalaReflection, dsl}
2929
import org.apache.spark.sql.catalyst.expressions._
3030
import org.apache.spark.sql.catalyst.types._
3131
import org.apache.spark.sql.catalyst.optimizer.Optimizer
32-
import org.apache.spark.sql.catalyst.plans.logical.{SetCommand, LogicalPlan}
32+
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
3333
import org.apache.spark.sql.catalyst.rules.RuleExecutor
3434
import org.apache.spark.sql.columnar.InMemoryRelation
3535
import org.apache.spark.sql.execution._
@@ -176,14 +176,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
176176
*
177177
* @group userf
178178
*/
179-
def sql(sqlText: String): SchemaRDD = {
180-
val result = new SchemaRDD(this, parseSql(sqlText))
181-
// We force query optimization to happen right away instead of letting it happen lazily like
182-
// when using the query DSL. This is so DDL commands behave as expected. This is only
183-
// generates the RDD lineage for DML queries, but do not perform any execution.
184-
result.queryExecution.toRdd
185-
result
186-
}
179+
def sql(sqlText: String): SchemaRDD = new SchemaRDD(this, parseSql(sqlText))
187180

188181
/** Returns the specified table as a SchemaRDD */
189182
def table(tableName: String): SchemaRDD =
@@ -288,8 +281,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
288281
protected[sql] val planner = new SparkPlanner
289282

290283
@transient
291-
protected[sql] lazy val emptyResult =
292-
sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1)
284+
protected[sql] lazy val emptyResult = sparkContext.parallelize(Seq.empty[Row], 1)
293285

294286
/**
295287
* Prepares a planned SparkPlan for execution by binding references to specific ordinals, and
@@ -309,35 +301,14 @@ class SQLContext(@transient val sparkContext: SparkContext)
309301
protected abstract class QueryExecution {
310302
def logical: LogicalPlan
311303

312-
def eagerlyProcess(plan: LogicalPlan): RDD[Row] = plan match {
313-
case SetCommand(key, value) =>
314-
// Only this case needs to be executed eagerly. The other cases will
315-
// be taken care of when the actual results are being extracted.
316-
// In the case of HiveContext, sqlConf is overridden to also pass the
317-
// pair into its HiveConf.
318-
if (key.isDefined && value.isDefined) {
319-
set(key.get, value.get)
320-
}
321-
// It doesn't matter what we return here, since this is only used
322-
// to force the evaluation to happen eagerly. To query the results,
323-
// one must use SchemaRDD operations to extract them.
324-
emptyResult
325-
case _ => executedPlan.execute()
326-
}
327-
328304
lazy val analyzed = analyzer(logical)
329305
lazy val optimizedPlan = optimizer(analyzed)
330306
// TODO: Don't just pick the first one...
331307
lazy val sparkPlan = planner(optimizedPlan).next()
332308
lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan)
333309

334310
/** Internal version of the RDD. Avoids copies and has no schema */
335-
lazy val toRdd: RDD[Row] = {
336-
logical match {
337-
case s: SetCommand => eagerlyProcess(s)
338-
case _ => executedPlan.execute()
339-
}
340-
}
311+
lazy val toRdd: RDD[Row] = executedPlan.execute()
341312

342313
protected def stringOrError[A](f: => A): String =
343314
try f.toString catch { case e: Throwable => e.toString }
@@ -359,7 +330,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
359330
* TODO: We only support primitive types, add support for nested types.
360331
*/
361332
private[sql] def inferSchema(rdd: RDD[Map[String, _]]): SchemaRDD = {
362-
val schema = rdd.first.map { case (fieldName, obj) =>
333+
val schema = rdd.first().map { case (fieldName, obj) =>
363334
val dataType = obj.getClass match {
364335
case c: Class[_] if c == classOf[java.lang.String] => StringType
365336
case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,7 @@ import java.util.{Map => JMap}
9999
@AlphaComponent
100100
class SchemaRDD(
101101
@transient val sqlContext: SQLContext,
102-
@transient protected[spark] val logicalPlan: LogicalPlan)
102+
@transient val baseLogicalPlan: LogicalPlan)
103103
extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike {
104104

105105
def baseSchemaRDD = this

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

Lines changed: 13 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,13 +20,14 @@ package org.apache.spark.sql
2020
import org.apache.spark.annotation.{DeveloperApi, Experimental}
2121
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
2222
import org.apache.spark.sql.catalyst.plans.logical._
23+
import org.apache.spark.sql.execution.SparkLogicalPlan
2324

2425
/**
2526
* Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java)
2627
*/
2728
private[sql] trait SchemaRDDLike {
2829
@transient val sqlContext: SQLContext
29-
@transient protected[spark] val logicalPlan: LogicalPlan
30+
@transient val baseLogicalPlan: LogicalPlan
3031

3132
private[sql] def baseSchemaRDD: SchemaRDD
3233

@@ -48,7 +49,17 @@ private[sql] trait SchemaRDDLike {
4849
*/
4950
@transient
5051
@DeveloperApi
51-
lazy val queryExecution = sqlContext.executePlan(logicalPlan)
52+
lazy val queryExecution = sqlContext.executePlan(baseLogicalPlan)
53+
54+
@transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match {
55+
// For various commands (like DDL) and queries with side effects, we force query optimization to
56+
// happen right away to let these side effects take place eagerly.
57+
case _: Command | _: InsertIntoTable | _: InsertIntoCreatedTable | _: WriteToFile =>
58+
queryExecution.toRdd
59+
SparkLogicalPlan(queryExecution.executedPlan)
60+
case _ =>
61+
baseLogicalPlan
62+
}
5263

5364
override def toString =
5465
s"""${super.toString}

sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ import org.apache.spark.storage.StorageLevel
3737
*/
3838
class JavaSchemaRDD(
3939
@transient val sqlContext: SQLContext,
40-
@transient protected[spark] val logicalPlan: LogicalPlan)
40+
@transient val baseLogicalPlan: LogicalPlan)
4141
extends JavaRDDLike[Row, JavaRDD[Row]]
4242
with SchemaRDDLike {
4343

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

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package org.apache.spark.sql.execution
1919

20-
import org.apache.spark.sql.{SQLConf, SQLContext, execution}
20+
import org.apache.spark.sql.{SQLContext, execution}
2121
import org.apache.spark.sql.catalyst.expressions._
2222
import org.apache.spark.sql.catalyst.planning._
2323
import org.apache.spark.sql.catalyst.plans._
@@ -157,7 +157,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
157157
InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil
158158
case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) =>
159159
InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil
160-
case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => {
160+
case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) =>
161161
val prunePushedDownFilters =
162162
if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) {
163163
(filters: Seq[Expression]) => {
@@ -186,7 +186,6 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
186186
filters,
187187
prunePushedDownFilters,
188188
ParquetTableScan(_, relation, filters)(sparkContext)) :: Nil
189-
}
190189

191190
case _ => Nil
192191
}
@@ -250,12 +249,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
250249
case class CommandStrategy(context: SQLContext) extends Strategy {
251250
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
252251
case logical.SetCommand(key, value) =>
253-
Seq(execution.SetCommandPhysical(key, value, plan.output)(context))
252+
Seq(execution.SetCommand(key, value, plan.output)(context))
254253
case logical.ExplainCommand(child) =>
255254
val executedPlan = context.executePlan(child).executedPlan
256-
Seq(execution.ExplainCommandPhysical(executedPlan, plan.output)(context))
255+
Seq(execution.ExplainCommand(executedPlan, plan.output)(context))
257256
case logical.CacheCommand(tableName, cache) =>
258-
Seq(execution.CacheCommandPhysical(tableName, cache)(context))
257+
Seq(execution.CacheCommand(tableName, cache)(context))
259258
case _ => Nil
260259
}
261260
}

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

Lines changed: 53 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -22,45 +22,69 @@ import org.apache.spark.rdd.RDD
2222
import org.apache.spark.sql.{SQLContext, Row}
2323
import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute}
2424

25+
trait Command {
26+
/**
27+
* A concrete command should override this lazy field to wrap up any side effects caused by the
28+
* command or any other computation that should be evaluated exactly once. The value of this field
29+
* can be used as the contents of the corresponding RDD generated from the physical plan of this
30+
* command.
31+
*
32+
* The `execute()` method of all the physical command classes should reference `sideEffectResult`
33+
* so that the command can be executed eagerly right after the command query is created.
34+
*/
35+
protected[sql] lazy val sideEffectResult: Seq[Any] = Seq.empty[Any]
36+
}
37+
2538
/**
2639
* :: DeveloperApi ::
2740
*/
2841
@DeveloperApi
29-
case class SetCommandPhysical(key: Option[String], value: Option[String], output: Seq[Attribute])
30-
(@transient context: SQLContext) extends LeafNode {
31-
def execute(): RDD[Row] = (key, value) match {
32-
// Set value for key k; the action itself would
33-
// have been performed in QueryExecution eagerly.
34-
case (Some(k), Some(v)) => context.emptyResult
42+
case class SetCommand(
43+
key: Option[String], value: Option[String], output: Seq[Attribute])(
44+
@transient context: SQLContext)
45+
extends LeafNode with Command {
46+
47+
override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match {
48+
// Set value for key k.
49+
case (Some(k), Some(v)) =>
50+
context.set(k, v)
51+
Array(k -> v)
52+
3553
// Query the value bound to key k.
36-
case (Some(k), None) =>
37-
val resultString = context.getOption(k) match {
38-
case Some(v) => s"$k=$v"
39-
case None => s"$k is undefined"
40-
}
41-
context.sparkContext.parallelize(Seq(new GenericRow(Array[Any](resultString))), 1)
54+
case (Some(k), _) =>
55+
Array(k -> context.getOption(k).getOrElse("<undefined>"))
56+
4257
// Query all key-value pairs that are set in the SQLConf of the context.
4358
case (None, None) =>
44-
val pairs = context.getAll
45-
val rows = pairs.map { case (k, v) =>
46-
new GenericRow(Array[Any](s"$k=$v"))
47-
}.toSeq
48-
// Assume config parameters can fit into one split (machine) ;)
49-
context.sparkContext.parallelize(rows, 1)
50-
// The only other case is invalid semantics and is impossible.
51-
case _ => context.emptyResult
59+
context.getAll
60+
61+
case _ =>
62+
throw new IllegalArgumentException()
5263
}
64+
65+
def execute(): RDD[Row] = {
66+
val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) }
67+
context.sparkContext.parallelize(rows, 1)
68+
}
69+
70+
override def otherCopyArgs = context :: Nil
5371
}
5472

5573
/**
5674
* :: DeveloperApi ::
5775
*/
5876
@DeveloperApi
59-
case class ExplainCommandPhysical(child: SparkPlan, output: Seq[Attribute])
60-
(@transient context: SQLContext) extends UnaryNode {
77+
case class ExplainCommand(
78+
child: SparkPlan, output: Seq[Attribute])(
79+
@transient context: SQLContext)
80+
extends UnaryNode with Command {
81+
82+
// Actually "EXPLAIN" command doesn't cause any side effect.
83+
override protected[sql] lazy val sideEffectResult: Seq[String] = this.toString.split("\n")
84+
6185
def execute(): RDD[Row] = {
62-
val planString = new GenericRow(Array[Any](child.toString))
63-
context.sparkContext.parallelize(Seq(planString))
86+
val explanation = sideEffectResult.mkString("\n")
87+
context.sparkContext.parallelize(Seq(new GenericRow(Array[Any](explanation))), 1)
6488
}
6589

6690
override def otherCopyArgs = context :: Nil
@@ -70,19 +94,20 @@ case class ExplainCommandPhysical(child: SparkPlan, output: Seq[Attribute])
7094
* :: DeveloperApi ::
7195
*/
7296
@DeveloperApi
73-
case class CacheCommandPhysical(tableName: String, doCache: Boolean)(@transient context: SQLContext)
74-
extends LeafNode {
97+
case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext)
98+
extends LeafNode with Command {
7599

76-
lazy val commandSideEffect = {
100+
override protected[sql] lazy val sideEffectResult = {
77101
if (doCache) {
78102
context.cacheTable(tableName)
79103
} else {
80104
context.uncacheTable(tableName)
81105
}
106+
Seq.empty[Any]
82107
}
83108

84109
override def execute(): RDD[Row] = {
85-
commandSideEffect
110+
sideEffectResult
86111
context.emptyResult
87112
}
88113

0 commit comments

Comments
 (0)