Skip to content

Commit 4a38c9b

Browse files
committed
[SPARK-5193][SQL] Tighten up SQLContext API
1. Removed 2 implicits (logicalPlanToSparkQuery and baseRelationToSchemaRDD) 2. Moved extraStrategies into ExperimentalMethods. 3. Made private methods protected[sql] so they don't show up in javadocs.
1 parent d5eeb35 commit 4a38c9b

File tree

7 files changed

+68
-38
lines changed

7 files changed

+68
-38
lines changed
Lines changed: 36 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,36 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql
19+
20+
import org.apache.spark.annotation.Experimental
21+
22+
/**
23+
* Holder for experimental methods for the bravest. We make NO guarantee about the stability
24+
* regarding binary compatibility and source compatibility of methods here.
25+
*/
26+
@Experimental
27+
class ExperimentalMethods protected[sql](sqlContext: SQLContext) {
28+
29+
/**
30+
* Allows extra strategies to be injected into the query planner at runtime. Note this API
31+
* should be consider experimental and is not intended to be stable across releases.
32+
*/
33+
@Experimental
34+
var extraStrategies: Seq[Strategy] = Nil
35+
36+
}

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

Lines changed: 9 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor
3737
import org.apache.spark.sql.execution._
3838
import org.apache.spark.sql.json._
3939
import org.apache.spark.sql.parquet.ParquetRelation
40-
import org.apache.spark.sql.sources.{BaseRelation, DDLParser, DataSourceStrategy, LogicalRelation}
40+
import org.apache.spark.sql.sources.{DDLParser, DataSourceStrategy}
4141
import org.apache.spark.sql.types._
4242

4343
/**
@@ -59,7 +59,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
5959
self =>
6060

6161
// Note that this is a lazy val so we can override the default value in subclasses.
62-
private[sql] lazy val conf: SQLConf = new SQLConf
62+
protected[sql] lazy val conf: SQLConf = new SQLConf
6363

6464
/** Set Spark SQL configuration properties. */
6565
def setConf(props: Properties): Unit = conf.setConf(props)
@@ -117,15 +117,6 @@ class SQLContext(@transient val sparkContext: SparkContext)
117117
case _ =>
118118
}
119119

120-
/**
121-
* :: DeveloperApi ::
122-
* Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan
123-
* interface is considered internal, and thus not guaranteed to be stable. As a result, using
124-
* them directly is not recommended.
125-
*/
126-
@DeveloperApi
127-
implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan)
128-
129120
/**
130121
* Creates a SchemaRDD from an RDD of case classes.
131122
*
@@ -139,10 +130,6 @@ class SQLContext(@transient val sparkContext: SparkContext)
139130
new SchemaRDD(this, LogicalRDD(attributeSeq, rowRDD)(self))
140131
}
141132

142-
implicit def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = {
143-
logicalPlanToSparkQuery(LogicalRelation(baseRelation))
144-
}
145-
146133
/**
147134
* :: DeveloperApi ::
148135
* Creates a [[SchemaRDD]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD.
@@ -336,12 +323,10 @@ class SQLContext(@transient val sparkContext: SparkContext)
336323
new SchemaRDD(this, catalog.lookupRelation(Seq(tableName)))
337324

338325
/**
339-
* :: DeveloperApi ::
340-
* Allows extra strategies to be injected into the query planner at runtime. Note this API
341-
* should be consider experimental and is not intended to be stable across releases.
326+
* A collection of methods that are considered experimental, but can be used to hook into
327+
* the query planner for advanced functionalities.
342328
*/
343-
@DeveloperApi
344-
var extraStrategies: Seq[Strategy] = Nil
329+
val experimental: ExperimentalMethods = new ExperimentalMethods(this)
345330

346331
protected[sql] class SparkPlanner extends SparkStrategies {
347332
val sparkContext: SparkContext = self.sparkContext
@@ -353,7 +338,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
353338
def numPartitions = self.conf.numShufflePartitions
354339

355340
def strategies: Seq[Strategy] =
356-
extraStrategies ++ (
341+
experimental.extraStrategies ++ (
357342
DataSourceStrategy ::
358343
DDLStrategy ::
359344
TakeOrdered ::
@@ -479,14 +464,14 @@ class SQLContext(@transient val sparkContext: SparkContext)
479464
* have the same format as the one generated by `toString` in scala.
480465
* It is only used by PySpark.
481466
*/
482-
private[sql] def parseDataType(dataTypeString: String): DataType = {
467+
protected[sql] def parseDataType(dataTypeString: String): DataType = {
483468
DataType.fromJson(dataTypeString)
484469
}
485470

486471
/**
487472
* Apply a schema defined by the schemaString to an RDD. It is only used by PySpark.
488473
*/
489-
private[sql] def applySchemaToPythonRDD(
474+
protected[sql] def applySchemaToPythonRDD(
490475
rdd: RDD[Array[Any]],
491476
schemaString: String): SchemaRDD = {
492477
val schema = parseDataType(schemaString).asInstanceOf[StructType]
@@ -496,7 +481,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
496481
/**
497482
* Apply a schema defined by the schema to an RDD. It is only used by PySpark.
498483
*/
499-
private[sql] def applySchemaToPythonRDD(
484+
protected[sql] def applySchemaToPythonRDD(
500485
rdd: RDD[Array[Any]],
501486
schema: StructType): SchemaRDD = {
502487

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

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,11 @@ package org.apache.spark.sql.execution
2020
import org.apache.spark.Logging
2121
import org.apache.spark.annotation.DeveloperApi
2222
import org.apache.spark.rdd.RDD
23+
import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext}
2324
import org.apache.spark.sql.catalyst.errors.TreeNodeException
2425
import org.apache.spark.sql.catalyst.expressions.{Row, Attribute}
2526
import org.apache.spark.sql.catalyst.plans.logical
2627
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
27-
import org.apache.spark.sql.{SQLConf, SQLContext}
2828

2929
/**
3030
* A logical command that is executed for its side-effects. `RunnableCommand`s are
@@ -137,14 +137,12 @@ case class CacheTableCommand(
137137
isLazy: Boolean) extends RunnableCommand {
138138

139139
override def run(sqlContext: SQLContext) = {
140-
import sqlContext._
141-
142-
plan.foreach(_.registerTempTable(tableName))
143-
cacheTable(tableName)
140+
plan.foreach(p => new SchemaRDD(sqlContext, p).registerTempTable(tableName))
141+
sqlContext.cacheTable(tableName)
144142

145143
if (!isLazy) {
146144
// Performs eager caching
147-
table(tableName).count()
145+
sqlContext.table(tableName).count()
148146
}
149147

150148
Seq.empty[Row]

sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import scala.util.parsing.combinator.syntactical.StandardTokenParsers
2222
import scala.util.parsing.combinator.PackratParsers
2323

2424
import org.apache.spark.Logging
25-
import org.apache.spark.sql.SQLContext
25+
import org.apache.spark.sql.{SchemaRDD, SQLContext}
2626
import org.apache.spark.sql.catalyst.plans.logical._
2727
import org.apache.spark.sql.catalyst.SqlLexical
2828
import org.apache.spark.sql.execution.RunnableCommand
@@ -234,8 +234,7 @@ private [sql] case class CreateTempTableUsing(
234234

235235
def run(sqlContext: SQLContext) = {
236236
val resolved = ResolvedDataSource(sqlContext, userSpecifiedSchema, provider, options)
237-
238-
sqlContext.baseRelationToSchemaRDD(resolved.relation).registerTempTable(tableName)
237+
new SchemaRDD(sqlContext, LogicalRelation(resolved.relation)).registerTempTable(tableName)
239238
Seq.empty
240239
}
241240
}

sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala

Lines changed: 14 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,8 +17,11 @@
1717

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

20+
import scala.language.implicitConversions
21+
2022
import org.apache.spark.{SparkConf, SparkContext}
21-
import org.apache.spark.sql.{SQLConf, SQLContext}
23+
import org.apache.spark.sql.{SchemaRDD, SQLConf, SQLContext}
24+
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2225

2326
/** A SQLContext that can be used for local testing. */
2427
object TestSQLContext
@@ -29,7 +32,16 @@ object TestSQLContext
2932
new SparkConf().set("spark.sql.testkey", "true"))) {
3033

3134
/** Fewer partitions to speed up testing. */
32-
private[sql] override lazy val conf: SQLConf = new SQLConf {
35+
protected[sql] override lazy val conf: SQLConf = new SQLConf {
3336
override def numShufflePartitions: Int = this.getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt
3437
}
38+
39+
/**
40+
* Turn a logical plan into a SchemaRDD. This should be removed once we have an easier way to
41+
* construct SchemaRDD directly out of local data without relying on implicits.
42+
*/
43+
protected[sql] implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = {
44+
new SchemaRDD(this, plan)
45+
}
46+
3547
}

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) {
7171
class HiveContext(sc: SparkContext) extends SQLContext(sc) {
7272
self =>
7373

74-
private[sql] override lazy val conf: SQLConf = new SQLConf {
74+
protected[sql] override lazy val conf: SQLConf = new SQLConf {
7575
override def dialect: String = getConf(SQLConf.DIALECT, "hiveql")
7676
}
7777

@@ -348,7 +348,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
348348
val hivePlanner = new SparkPlanner with HiveStrategies {
349349
val hiveContext = self
350350

351-
override def strategies: Seq[Strategy] = extraStrategies ++ Seq(
351+
override def strategies: Seq[Strategy] = experimental.extraStrategies ++ Seq(
352352
DataSourceStrategy,
353353
HiveCommandStrategy(self),
354354
HiveDDLStrategy,

sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -102,7 +102,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
102102
new this.QueryExecution { val logical = plan }
103103

104104
/** Fewer partitions to speed up testing. */
105-
private[sql] override lazy val conf: SQLConf = new SQLConf {
105+
protected[sql] override lazy val conf: SQLConf = new SQLConf {
106106
override def numShufflePartitions: Int = getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt
107107
override def dialect: String = getConf(SQLConf.DIALECT, "hiveql")
108108
}

0 commit comments

Comments
 (0)