aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main
diff options
context:
space:
mode:
authorCheng Lian <lian.cs.zju@gmail.com>2014-10-01 16:00:29 -0700
committerMichael Armbrust <michael@databricks.com>2014-10-01 16:00:29 -0700
commita31f4ff22f98c01f0d9b7d1240080ff2689c1270 (patch)
tree87939d3db70e10f19ca3fefe437c619c085d8e00 /sql/core/src/main
parentf84b228c4002073ee4ff53be50462a63f48bd508 (diff)
downloadspark-a31f4ff22f98c01f0d9b7d1240080ff2689c1270.tar.gz
spark-a31f4ff22f98c01f0d9b7d1240080ff2689c1270.tar.bz2
spark-a31f4ff22f98c01f0d9b7d1240080ff2689c1270.zip
[SQL] Made Command.sideEffectResult protected
Considering `Command.executeCollect()` simply delegates to `Command.sideEffectResult`, we no longer need to leave the latter `protected[sql]`. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #2431 from liancheng/narrow-scope and squashes the following commits: 1bfc16a [Cheng Lian] Made Command.sideEffectResult protected
Diffstat (limited to 'sql/core/src/main')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala10
1 files changed, 5 insertions, 5 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
index c2f48a902a..f88099ec07 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala
@@ -37,7 +37,7 @@ trait Command {
* The `execute()` method of all the physical command classes should reference `sideEffectResult`
* so that the command can be executed eagerly right after the command query is created.
*/
- protected[sql] lazy val sideEffectResult: Seq[Row] = Seq.empty[Row]
+ protected lazy val sideEffectResult: Seq[Row] = Seq.empty[Row]
override def executeCollect(): Array[Row] = sideEffectResult.toArray
@@ -53,7 +53,7 @@ case class SetCommand(
@transient context: SQLContext)
extends LeafNode with Command with Logging {
- override protected[sql] lazy val sideEffectResult: Seq[Row] = (key, value) match {
+ override protected lazy val sideEffectResult: Seq[Row] = (key, value) match {
// Set value for key k.
case (Some(k), Some(v)) =>
if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) {
@@ -121,7 +121,7 @@ case class ExplainCommand(
extends LeafNode with Command {
// Run through the optimizer to generate the physical plan.
- override protected[sql] lazy val sideEffectResult: Seq[Row] = try {
+ override protected lazy val sideEffectResult: Seq[Row] = try {
// TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties.
val queryExecution = context.executePlan(logicalPlan)
val outputString = if (extended) queryExecution.toString else queryExecution.simpleString
@@ -141,7 +141,7 @@ case class ExplainCommand(
case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext)
extends LeafNode with Command {
- override protected[sql] lazy val sideEffectResult = {
+ override protected lazy val sideEffectResult = {
if (doCache) {
context.cacheTable(tableName)
} else {
@@ -161,7 +161,7 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])(
@transient context: SQLContext)
extends LeafNode with Command {
- override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+ override protected lazy val sideEffectResult: Seq[Row] = {
Row("# Registered as a temporary table", null, null) +:
child.output.map(field => Row(field.name, field.dataType.toString, null))
}