Skip to content

Commit 44fc0ff

Browse files
author
Andrew Or
committed
Move commands.scala to command package
1 parent 6250cf1 commit 44fc0ff

File tree

21 files changed

+29
-21
lines changed

21 files changed

+29
-21
lines changed

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,6 @@ package org.apache.spark.sql
2020
import java.io.CharArrayWriter
2121

2222
import scala.language.implicitConversions
23-
import scala.reflect.ClassTag
2423
import scala.reflect.runtime.universe.TypeTag
2524

2625
import com.fasterxml.jackson.core.JsonFactory
@@ -37,7 +36,8 @@ import org.apache.spark.sql.catalyst.optimizer.CombineUnions
3736
import org.apache.spark.sql.catalyst.plans._
3837
import org.apache.spark.sql.catalyst.plans.logical._
3938
import org.apache.spark.sql.catalyst.util.usePrettyExpression
40-
import org.apache.spark.sql.execution.{ExplainCommand, FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution}
39+
import org.apache.spark.sql.execution.{FileRelation, LogicalRDD, Queryable, QueryExecution, SQLExecution}
40+
import org.apache.spark.sql.execution.command.ExplainCommand
4141
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation}
4242
import org.apache.spark.sql.execution.datasources.json.JacksonGenerator
4343
import org.apache.spark.sql.execution.python.EvaluatePython

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

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.optimizer.Optimizer
3838
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Range}
3939
import org.apache.spark.sql.catalyst.rules.RuleExecutor
4040
import org.apache.spark.sql.execution._
41+
import org.apache.spark.sql.execution.command.ShowTablesCommand
4142
import org.apache.spark.sql.execution.datasources._
4243
import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab}
4344
import org.apache.spark.sql.internal.{SessionState, SQLConf}

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.{CatalystQl, TableIdentifier}
2121
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
2222
import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, SimpleParserConf}
2323
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
24-
import org.apache.spark.sql.catalyst.plans.logical
24+
import org.apache.spark.sql.execution.command._
2525
import org.apache.spark.sql.execution.datasources._
2626
import org.apache.spark.sql.types.StructType
2727

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,8 +27,8 @@ import org.apache.spark.sql.catalyst.plans._
2727
import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, LogicalPlan}
2828
import org.apache.spark.sql.catalyst.plans.physical._
2929
import org.apache.spark.sql.execution
30-
import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand}
3130
import org.apache.spark.sql.execution.columnar.{InMemoryColumnarTableScan, InMemoryRelation}
31+
import org.apache.spark.sql.execution.command.{DescribeCommand => RunnableDescribeCommand, _}
3232
import org.apache.spark.sql.execution.datasources.{CreateTableUsing, CreateTempTableUsing, DescribeCommand => LogicalDescribeCommand, _}
3333
import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight}
3434
import org.apache.spark.sql.internal.SQLConf

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

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,7 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql.execution
18+
package org.apache.spark.sql.execution.command
1919

2020
import java.util.NoSuchElementException
2121

@@ -27,9 +27,11 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException
2727
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
2828
import org.apache.spark.sql.catalyst.plans.logical
2929
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
30+
import org.apache.spark.sql.execution.SparkPlan
3031
import org.apache.spark.sql.internal.SQLConf
3132
import org.apache.spark.sql.types._
3233

34+
3335
/**
3436
* A logical command that is executed for its side-effects. `RunnableCommand`s are
3537
* wrapped in `ExecutedCommand` during execution.

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
3232
import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
3333
import org.apache.spark.sql.execution.PhysicalRDD.{INPUT_PATHS, PUSHED_FILTERS}
3434
import org.apache.spark.sql.execution.SparkPlan
35+
import org.apache.spark.sql.execution.command.ExecutedCommand
3536
import org.apache.spark.sql.sources._
3637
import org.apache.spark.sql.types.{StringType, StructType}
3738
import org.apache.spark.unsafe.types.UTF8String
@@ -146,12 +147,12 @@ private[sql] object DataSourceStrategy extends Strategy with Logging {
146147

147148
case i @ logical.InsertIntoTable(l @ LogicalRelation(t: InsertableRelation, _, _),
148149
part, query, overwrite, false) if part.isEmpty =>
149-
execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil
150+
ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil
150151

151152
case i @ logical.InsertIntoTable(
152153
l @ LogicalRelation(t: HadoopFsRelation, _, _), part, query, overwrite, false) =>
153154
val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append
154-
execution.ExecutedCommand(InsertIntoHadoopFsRelation(t, query, mode)) :: Nil
155+
ExecutedCommand(InsertIntoHadoopFsRelation(t, query, mode)) :: Nil
155156

156157
case _ => Nil
157158
}

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources
1919

2020
import org.apache.spark.sql._
2121
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
22-
import org.apache.spark.sql.execution.RunnableCommand
22+
import org.apache.spark.sql.execution.command.RunnableCommand
2323
import org.apache.spark.sql.sources.InsertableRelation
2424

2525

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,8 @@ import org.apache.spark.sql._
2828
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
2929
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
3030
import org.apache.spark.sql.catalyst.InternalRow
31-
import org.apache.spark.sql.execution.{RunnableCommand, SQLExecution}
31+
import org.apache.spark.sql.execution.SQLExecution
32+
import org.apache.spark.sql.execution.command.RunnableCommand
3233
import org.apache.spark.sql.internal.SQLConf
3334
import org.apache.spark.sql.sources._
3435
import org.apache.spark.util.Utils

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
2222
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
2323
import org.apache.spark.sql.catalyst.plans.logical
2424
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
25-
import org.apache.spark.sql.execution.RunnableCommand
25+
import org.apache.spark.sql.execution.command.RunnableCommand
2626
import org.apache.spark.sql.types._
2727

2828
/**

sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,10 +24,10 @@ import java.util.{Calendar, GregorianCalendar, Properties}
2424
import org.h2.jdbc.JdbcSQLException
2525
import org.scalatest.{BeforeAndAfter, PrivateMethodTester}
2626

27-
import org.apache.spark.sql.execution.ExplainCommand
2827
import org.apache.spark.SparkFunSuite
2928
import org.apache.spark.sql.{DataFrame, Row}
3029
import org.apache.spark.sql.execution.PhysicalRDD
30+
import org.apache.spark.sql.execution.command.ExplainCommand
3131
import org.apache.spark.sql.execution.datasources.LogicalRelation
3232
import org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD
3333
import org.apache.spark.sql.sources._

0 commit comments

Comments
 (0)