Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 5 additions & 5 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import scala.collection.immutable
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal

import org.apache.spark.{SparkException, SparkContext}
import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.rdd.RDD
Expand All @@ -45,9 +44,10 @@ import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab}
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.types._
import org.apache.spark.sql.{execution => sparkexecution}
import org.apache.spark.sql.util.ExecutionListenerManager
import org.apache.spark.sql.{execution => sparkexecution}
import org.apache.spark.util.Utils
import org.apache.spark.{SparkContext, SparkException}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

super nit: import order

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

How to define this order in Intelij?


/**
* The entry point for working with structured data (rows and columns) in Spark. Allows the
Expand Down Expand Up @@ -401,7 +401,7 @@ class SQLContext private[sql](
*/
@Experimental
def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = {
SparkPlan.currentContext.set(self)
SQLContext.setActive(self)
val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
val attributeSeq = schema.toAttributes
val rowRDD = RDDConversions.productToRowRdd(rdd, schema.map(_.dataType))
Expand All @@ -417,7 +417,7 @@ class SQLContext private[sql](
*/
@Experimental
def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = {
SparkPlan.currentContext.set(self)
SQLContext.setActive(self)
val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
val attributeSeq = schema.toAttributes
DataFrame(self, LocalRelation.fromProduct(attributeSeq, data))
Expand Down Expand Up @@ -1328,7 +1328,7 @@ object SQLContext {
activeContext.remove()
}

private[sql] def getActiveContextOption(): Option[SQLContext] = {
private[sql] def getActive(): Option[SQLContext] = {
Option(activeContext.get())
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,9 +42,8 @@ class QueryExecution(val sqlContext: SQLContext, val logical: LogicalPlan) {

lazy val optimizedPlan: LogicalPlan = sqlContext.optimizer.execute(withCachedData)

// TODO: Don't just pick the first one...
lazy val sparkPlan: SparkPlan = {
SparkPlan.currentContext.set(sqlContext)
SQLContext.setActive(sqlContext)
sqlContext.planner.plan(optimizedPlan).next()
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,21 +23,15 @@ import scala.collection.mutable.ArrayBuffer

import org.apache.spark.Logging
import org.apache.spark.rdd.{RDD, RDDOperationScope}
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.{Row, SQLContext}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.execution.metric.{LongSQLMetric, SQLMetric}
import org.apache.spark.sql.types.DataType

object SparkPlan {
protected[sql] val currentContext = new ThreadLocal[SQLContext]()
}

/**
* The base class for physical operators.
*/
Expand All @@ -49,7 +43,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ
* populated by the query planning infrastructure.
*/
@transient
protected[spark] final val sqlContext = SparkPlan.currentContext.get()
protected[spark] final val sqlContext = SQLContext.getActive().get

protected def sparkContext = sqlContext.sparkContext

Expand All @@ -69,7 +63,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ

/** Overridden make copy also propogates sqlContext to copied plan. */
override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = {
SparkPlan.currentContext.set(sqlContext)
SQLContext.setActive(sqlContext)
super.makeCopy(newArgs)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ class MultiSQLContextsSuite extends SparkFunSuite with BeforeAndAfterAll {
private var sparkConf: SparkConf = _

override protected def beforeAll(): Unit = {
originalActiveSQLContext = SQLContext.getActiveContextOption()
originalActiveSQLContext = SQLContext.getActive()
originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption()

SQLContext.clearActive()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll {
private var originalInstantiatedSQLContext: Option[SQLContext] = _

override protected def beforeAll(): Unit = {
originalActiveSQLContext = SQLContext.getActiveContextOption()
originalActiveSQLContext = SQLContext.getActive()
originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption()

SQLContext.clearActive()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.execution

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.Row
import org.apache.spark.sql.{SQLContext, Row}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Attribute, Literal, IsNull}
import org.apache.spark.sql.catalyst.util.GenericArrayData
Expand Down Expand Up @@ -94,7 +94,7 @@ class RowFormatConvertersSuite extends SparkPlanTest with SharedSQLContext {
}

test("SPARK-9683: copy UTF8String when convert unsafe array/map to safe") {
SparkPlan.currentContext.set(sqlContext)
SQLContext.setActive(sqlContext)
val schema = ArrayType(StringType)
val rows = (1 to 100).map { i =>
InternalRow(new GenericArrayData(Array[Any](UTF8String.fromString(i.toString))))
Expand Down