From a31f4ff22f98c01f0d9b7d1240080ff2689c1270 Mon Sep 17 00:00:00 2001
From: Cheng Lian <lian.cs.zju@gmail.com>
Date: Wed, 1 Oct 2014 16:00:29 -0700
Subject: [PATCH] [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
---
 .../apache/spark/sql/execution/commands.scala    | 10 +++++-----
 .../org/apache/spark/sql/hive/HiveContext.scala  |  2 +-
 .../sql/hive/execution/CreateTableAsSelect.scala | 16 ++++++++--------
 .../execution/DescribeHiveTableCommand.scala     |  2 +-
 .../spark/sql/hive/execution/NativeCommand.scala |  2 +-
 .../spark/sql/hive/execution/commands.scala      |  6 +++---
 6 files changed, 19 insertions(+), 19 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))
   }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 3e1a7b7152..20ebe4996c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -404,7 +404,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
         // be similar with Hive.
         describeHiveTableCommand.hiveString
       case command: PhysicalCommand =>
-        command.sideEffectResult.map(_.head.toString)
+        command.executeCollect().map(_.head.toString)
 
       case other =>
         val result: Seq[Seq[Any]] = toRdd.collect().toSeq
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
index 1017fe6d53..3625708d03 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala
@@ -30,23 +30,23 @@ import org.apache.spark.sql.hive.MetastoreRelation
  * Create table and insert the query result into it.
  * @param database the database name of the new relation
  * @param tableName the table name of the new relation
- * @param insertIntoRelation function of creating the `InsertIntoHiveTable` 
+ * @param insertIntoRelation function of creating the `InsertIntoHiveTable`
  *        by specifying the `MetaStoreRelation`, the data will be inserted into that table.
  * TODO Add more table creating properties,  e.g. SerDe, StorageHandler, in-memory cache etc.
  */
 @Experimental
 case class CreateTableAsSelect(
-  database: String,
-  tableName: String,
-  query: SparkPlan,
-  insertIntoRelation: MetastoreRelation => InsertIntoHiveTable)
-    extends LeafNode with Command {
+    database: String,
+    tableName: String,
+    query: SparkPlan,
+    insertIntoRelation: MetastoreRelation => InsertIntoHiveTable)
+  extends LeafNode with Command {
 
   def output = Seq.empty
 
   // A lazy computing of the metastoreRelation
   private[this] lazy val metastoreRelation: MetastoreRelation = {
-    // Create the table 
+    // Create the table
     val sc = sqlContext.asInstanceOf[HiveContext]
     sc.catalog.createTable(database, tableName, query.output, false)
     // Get the Metastore Relation
@@ -55,7 +55,7 @@ case class CreateTableAsSelect(
     }
   }
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  override protected lazy val sideEffectResult: Seq[Row] = {
     insertIntoRelation(metastoreRelation).execute
     Seq.empty[Row]
   }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
index 317801001c..106cede978 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala
@@ -48,7 +48,7 @@ case class DescribeHiveTableCommand(
         .mkString("\t")
   }
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  override protected lazy val sideEffectResult: Seq[Row] = {
     // Trying to mimic the format of Hive's output. But not exactly the same.
     var results: Seq[(String, String, String)] = Nil
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala
index 8f10e1ba7f..6930c2babd 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala
@@ -32,7 +32,7 @@ case class NativeCommand(
     @transient context: HiveContext)
   extends LeafNode with Command {
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_))
+  override protected lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_))
 
   override def otherCopyArgs = context :: Nil
 }
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
index d61c5e274a..0fc674af31 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala
@@ -37,7 +37,7 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command {
 
   def output = Seq.empty
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  override protected lazy val sideEffectResult: Seq[Row] = {
     hiveContext.analyze(tableName)
     Seq.empty[Row]
   }
@@ -53,7 +53,7 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with
 
   def output = Seq.empty
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  override protected lazy val sideEffectResult: Seq[Row] = {
     val ifExistsClause = if (ifExists) "IF EXISTS " else ""
     hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName")
     hiveContext.catalog.unregisterTable(None, tableName)
@@ -70,7 +70,7 @@ case class AddJar(path: String) extends LeafNode with Command {
 
   override def output = Seq.empty
 
-  override protected[sql] lazy val sideEffectResult: Seq[Row] = {
+  override protected lazy val sideEffectResult: Seq[Row] = {
     hiveContext.runSqlHive(s"ADD JAR $path")
     hiveContext.sparkContext.addJar(path)
     Seq.empty[Row]
-- 
GitLab