Skip to content

Commit f9aa88d

Browse files
committed
[SPARK-9763][SQL] Minimize exposure of internal SQL classes.
1 parent 00b655c commit f9aa88d

File tree

74 files changed

+1077
-897
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

74 files changed

+1077
-897
lines changed
Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,3 @@
1-
org.apache.spark.sql.jdbc.DefaultSource
2-
org.apache.spark.sql.json.DefaultSource
3-
org.apache.spark.sql.parquet.DefaultSource
1+
org.apache.spark.sql.execution.datasources.jdbc.DefaultSource
2+
org.apache.spark.sql.execution.datasources.json.DefaultSource
3+
org.apache.spark.sql.execution.datasources.parquet.DefaultSource

sql/core/src/main/resources/org/apache/spark/sql/ui/static/spark-sql-viz.css renamed to sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.css

File renamed without changes.

sql/core/src/main/resources/org/apache/spark/sql/ui/static/spark-sql-viz.js renamed to sql/core/src/main/resources/org/apache/spark/sql/execution/ui/static/spark-sql-viz.js

File renamed without changes.

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
3939
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
4040
import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD, SQLExecution}
4141
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, LogicalRelation}
42-
import org.apache.spark.sql.json.JacksonGenerator
42+
import org.apache.spark.sql.execution.datasources.json.JacksonGenerator
4343
import org.apache.spark.sql.sources.HadoopFsRelation
4444
import org.apache.spark.sql.types._
4545
import org.apache.spark.storage.StorageLevel

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -25,10 +25,10 @@ import org.apache.spark.annotation.Experimental
2525
import org.apache.spark.api.java.JavaRDD
2626
import org.apache.spark.deploy.SparkHadoopUtil
2727
import org.apache.spark.rdd.RDD
28+
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation}
29+
import org.apache.spark.sql.execution.datasources.json.JSONRelation
30+
import org.apache.spark.sql.execution.datasources.parquet.ParquetRelation
2831
import org.apache.spark.sql.execution.datasources.{LogicalRelation, ResolvedDataSource}
29-
import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation}
30-
import org.apache.spark.sql.json.JSONRelation
31-
import org.apache.spark.sql.parquet.ParquetRelation
3232
import org.apache.spark.sql.types.StructType
3333
import org.apache.spark.{Logging, Partition}
3434

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

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,8 @@ import org.apache.spark.annotation.Experimental
2323
import org.apache.spark.sql.catalyst.{SqlParser, TableIdentifier}
2424
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
2525
import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable
26+
import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils
2627
import org.apache.spark.sql.execution.datasources.{CreateTableUsingAsSelect, ResolvedDataSource}
27-
import org.apache.spark.sql.jdbc.{JDBCWriteDetails, JdbcUtils}
2828
import org.apache.spark.sql.sources.HadoopFsRelation
2929

3030

@@ -264,15 +264,15 @@ final class DataFrameWriter private[sql](df: DataFrame) {
264264

265265
// Create the table if the table didn't exist.
266266
if (!tableExists) {
267-
val schema = JDBCWriteDetails.schemaString(df, url)
267+
val schema = JdbcUtils.schemaString(df, url)
268268
val sql = s"CREATE TABLE $table ($schema)"
269269
conn.prepareStatement(sql).executeUpdate()
270270
}
271271
} finally {
272272
conn.close()
273273
}
274274

275-
JDBCWriteDetails.saveTable(df, url, table, connectionProperties)
275+
JdbcUtils.saveTable(df, url, table, connectionProperties)
276276
}
277277

278278
/**

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,7 @@ import org.apache.spark.sql.execution._
4343
import org.apache.spark.sql.execution.datasources._
4444
import org.apache.spark.sql.sources.BaseRelation
4545
import org.apache.spark.sql.types._
46-
import org.apache.spark.sql.ui.{SQLListener, SQLTab}
46+
import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab}
4747
import org.apache.spark.unsafe.types.UTF8String
4848
import org.apache.spark.util.Utils
4949

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicLong
2121

2222
import org.apache.spark.SparkContext
2323
import org.apache.spark.sql.SQLContext
24-
import org.apache.spark.sql.ui.SparkPlanGraph
24+
import org.apache.spark.sql.execution.ui.SparkPlanGraph
2525
import org.apache.spark.util.Utils
2626

2727
private[sql] object SQLExecution {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,7 +32,7 @@ import org.apache.spark.sql.Row
3232
import org.apache.spark.sql.catalyst.expressions.codegen._
3333
import org.apache.spark.sql.catalyst.plans.QueryPlan
3434
import org.apache.spark.sql.catalyst.plans.physical._
35-
import org.apache.spark.sql.metric.{IntSQLMetric, LongSQLMetric, SQLMetric, SQLMetrics}
35+
import org.apache.spark.sql.execution.metric.{IntSQLMetric, LongSQLMetric, SQLMetric, SQLMetrics}
3636
import org.apache.spark.sql.types.DataType
3737

3838
object SparkPlan {

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.CatalystTypeConverters
2626
import org.apache.spark.sql.catalyst.errors._
2727
import org.apache.spark.sql.catalyst.expressions._
2828
import org.apache.spark.sql.catalyst.plans.physical._
29-
import org.apache.spark.sql.metric.SQLMetrics
29+
import org.apache.spark.sql.execution.metric.SQLMetrics
3030
import org.apache.spark.sql.types.StructType
3131
import org.apache.spark.util.collection.ExternalSorter
3232
import org.apache.spark.util.collection.unsafe.sort.PrefixComparator

0 commit comments

Comments
 (0)