Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
58 commits
Select commit Hold shift + click to select a range
16ca7ac
It compiles!!
Aug 4, 2015
5e6a20c
Refactor SQLTestUtils to reduce duplication
Aug 5, 2015
d1d1449
Remove HiveTest singleton
Aug 5, 2015
d4aafb1
Avoid the need to switch to HiveContexts
Aug 5, 2015
6345cee
Clean up JsonSuite
Aug 5, 2015
68ac6fe
Rename the test traits properly
Aug 5, 2015
b15fdc6
Stop SparkContext in Java SQL tests
Aug 5, 2015
0d74a72
Load test data early in case tables are accessed by name
Aug 5, 2015
eee415d
Refactor implicits into SQLTestUtils
Aug 10, 2015
55d0b1b
Fix Java not serializable exception in tests
Aug 10, 2015
4f59bee
Fix DataSourceTest et al.
Aug 10, 2015
88d4f16
Fix hive tests to use the same pattern
Aug 11, 2015
5fe4bfb
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
c4a22bc
Fix compile after resolving merge conflicts
Aug 11, 2015
d9a8390
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
f5619f8
Fix test compile after resolving merge conflicts
Aug 11, 2015
9395cfa
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
c51b3d8
Fix OuterJoinSuite
Aug 11, 2015
997715e
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
1400770
Fix style
Aug 11, 2015
c92a3b0
Fix MiMa
Aug 11, 2015
4debedf
Clean up inheritance in test util traits
Aug 11, 2015
ca87dda
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 11, 2015
1cf53ad
Create new context in SBT console by default
Aug 11, 2015
bc5c999
Fix SemiJoinSuite
Aug 12, 2015
19fd6c3
Fix InnerJoinSuite
Aug 12, 2015
e1e513e
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
1e4c321
Address comments
Aug 12, 2015
54848e2
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
94f9c77
Revert the removal of some BeforeAndAfters
Aug 12, 2015
bec7d28
Fix a hive test + minor format updates
Aug 12, 2015
24c086d
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
561eacc
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
aaba277
Fix places where we override before / after alls
Aug 12, 2015
aac8f7f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
40959bb
Fix test
Aug 12, 2015
55d6624
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 12, 2015
4ed58c8
Revert a few merge-conflict-induced unintentional changes
Aug 12, 2015
0ce5638
Minor updates
Aug 13, 2015
814df2f
Add back singletons but deprecate them
Aug 13, 2015
099c74f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
451fa37
Use consistent name for added back singletons
Aug 13, 2015
9ea7f7c
Fix style
Aug 13, 2015
ddc4b05
Revert "Use consistent name for added back singletons"
Aug 13, 2015
ece3a81
Remove TestSQLContext, but keep TestHive
Aug 13, 2015
94cc3b5
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
8d69bf8
Fix test
Aug 13, 2015
7a0b1ef
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
b58ae73
Fix tests
Aug 13, 2015
828144f
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
c4d44c9
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
48af8e4
Fix another before / after alls
Aug 13, 2015
0606c82
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
f599bbc
Revert all Hive related changes
Aug 13, 2015
aed7fc7
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
d85a6d8
Add a shorthand for sqlContext.sql / ctx.sql
Aug 13, 2015
0b60325
Fix hive test compile
Aug 13, 2015
821ea67
Merge branch 'master' of github.com:apache/spark into sql-tests-refactor
Aug 13, 2015
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: 10 additions & 0 deletions project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,16 @@ object MimaExcludes {
// SPARK-4751 Dynamic allocation for standalone mode
ProblemFilters.exclude[MissingMethodProblem](
"org.apache.spark.SparkContext.supportDynamicAllocation")
) ++ Seq(
// SPARK-9580: Remove SQL test singletons
ProblemFilters.exclude[MissingClassProblem](
"org.apache.spark.sql.test.LocalSQLContext$SQLSession"),
ProblemFilters.exclude[MissingClassProblem](
"org.apache.spark.sql.test.LocalSQLContext"),
ProblemFilters.exclude[MissingClassProblem](
"org.apache.spark.sql.test.TestSQLContext"),
ProblemFilters.exclude[MissingClassProblem](
"org.apache.spark.sql.test.TestSQLContext$")
) ++ Seq(
// SPARK-9704 Made ProbabilisticClassifier, Identifiable, VectorUDT public APIs
ProblemFilters.exclude[IncompatibleResultTypeProblem](
Expand Down
16 changes: 11 additions & 5 deletions project/SparkBuild.scala
Original file line number Diff line number Diff line change
Expand Up @@ -319,6 +319,8 @@ object SQL {
lazy val settings = Seq(
initialCommands in console :=
"""
|import org.apache.spark.SparkContext
|import org.apache.spark.sql.SQLContext
|import org.apache.spark.sql.catalyst.analysis._
|import org.apache.spark.sql.catalyst.dsl._
|import org.apache.spark.sql.catalyst.errors._
Expand All @@ -328,9 +330,14 @@ object SQL {
|import org.apache.spark.sql.catalyst.util._
|import org.apache.spark.sql.execution
|import org.apache.spark.sql.functions._
|import org.apache.spark.sql.test.TestSQLContext._
|import org.apache.spark.sql.types._""".stripMargin,
cleanupCommands in console := "sparkContext.stop()"
|import org.apache.spark.sql.types._
|
|val sc = new SparkContext("local[*]", "dev-shell")
|val sqlContext = new SQLContext(sc)
|import sqlContext.implicits._
|import sqlContext._
""".stripMargin,
cleanupCommands in console := "sc.stop()"
)
}

Expand All @@ -340,15 +347,14 @@ object Hive {
javaOptions += "-XX:MaxPermSize=256m",
// Specially disable assertions since some Hive tests fail them
javaOptions in Test := (javaOptions in Test).value.filterNot(_ == "-ea"),
// Multiple queries rely on the TestHive singleton. See comments there for more details.
parallelExecution in Test := false,
// Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings
// only for this subproject.
scalacOptions <<= scalacOptions map { currentOpts: Seq[String] =>
currentOpts.filterNot(_ == "-deprecation")
},
initialCommands in console :=
"""
|import org.apache.spark.SparkContext
|import org.apache.spark.sql.catalyst.analysis._
|import org.apache.spark.sql.catalyst.dsl._
|import org.apache.spark.sql.catalyst.errors._
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,10 @@

package org.apache.spark.sql.catalyst.analysis

import org.scalatest.BeforeAndAfter

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.Inner
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.types._
Expand All @@ -42,7 +38,7 @@ case class UnresolvedTestPlan() extends LeafNode {
override def output: Seq[Attribute] = Nil
}

class AnalysisErrorSuite extends AnalysisTest with BeforeAndAfter {
class AnalysisErrorSuite extends AnalysisTest {
import TestRelations._

def errorTest(
Expand Down
97 changes: 4 additions & 93 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.AtomicReference

import scala.collection.JavaConversions._
import scala.collection.immutable
import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal

Expand All @@ -41,10 +40,9 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.{InternalRow, ParserDialect, _}
import org.apache.spark.sql.execution._
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.ui.{SQLListener, SQLTab}
import org.apache.spark.unsafe.types.UTF8String
import org.apache.spark.util.Utils

/**
Expand Down Expand Up @@ -334,97 +332,10 @@ class SQLContext(@transient val sparkContext: SparkContext)
* @since 1.3.0
*/
@Experimental
object implicits extends Serializable {
// scalastyle:on

/**
* Converts $"col name" into an [[Column]].
* @since 1.3.0
*/
implicit class StringToColumn(val sc: StringContext) {
def $(args: Any*): ColumnName = {
new ColumnName(sc.s(args: _*))
}
}

/**
* An implicit conversion that turns a Scala `Symbol` into a [[Column]].
* @since 1.3.0
*/
implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name)

/**
* Creates a DataFrame from an RDD of case classes or tuples.
* @since 1.3.0
*/
implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = {
DataFrameHolder(self.createDataFrame(rdd))
}

/**
* Creates a DataFrame from a local Seq of Product.
* @since 1.3.0
*/
implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder =
{
DataFrameHolder(self.createDataFrame(data))
}

// Do NOT add more implicit conversions. They are likely to break source compatibility by
// making existing implicit conversions ambiguous. In particular, RDD[Double] is dangerous
// because of [[DoubleRDDFunctions]].

/**
* Creates a single column DataFrame from an RDD[Int].
* @since 1.3.0
*/
implicit def intRddToDataFrameHolder(data: RDD[Int]): DataFrameHolder = {
val dataType = IntegerType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.setInt(0, v)
row: InternalRow
}
}
DataFrameHolder(
self.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}

/**
* Creates a single column DataFrame from an RDD[Long].
* @since 1.3.0
*/
implicit def longRddToDataFrameHolder(data: RDD[Long]): DataFrameHolder = {
val dataType = LongType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.setLong(0, v)
row: InternalRow
}
}
DataFrameHolder(
self.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}

/**
* Creates a single column DataFrame from an RDD[String].
* @since 1.3.0
*/
implicit def stringRddToDataFrameHolder(data: RDD[String]): DataFrameHolder = {
val dataType = StringType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.update(0, UTF8String.fromString(v))
row: InternalRow
}
}
DataFrameHolder(
self.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}
object implicits extends SQLImplicits with Serializable {
protected override def _sqlContext: SQLContext = self
}
// scalastyle:on

/**
* :: Experimental ::
Expand Down
123 changes: 123 additions & 0 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql

import scala.language.implicitConversions
import scala.reflect.runtime.universe.TypeTag

import org.apache.spark.rdd.RDD
import org.apache.spark.sql.types._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.SpecificMutableRow
import org.apache.spark.sql.types.StructField
import org.apache.spark.unsafe.types.UTF8String

/**
* A collection of implicit methods for converting common Scala objects into [[DataFrame]]s.
*/
private[sql] abstract class SQLImplicits {
protected def _sqlContext: SQLContext

/**
* Converts $"col name" into an [[Column]].
* @since 1.3.0
*/
implicit class StringToColumn(val sc: StringContext) {
def $(args: Any*): ColumnName = {
new ColumnName(sc.s(args: _*))
}
}

/**
* An implicit conversion that turns a Scala `Symbol` into a [[Column]].
* @since 1.3.0
*/
implicit def symbolToColumn(s: Symbol): ColumnName = new ColumnName(s.name)

/**
* Creates a DataFrame from an RDD of case classes or tuples.
* @since 1.3.0
*/
implicit def rddToDataFrameHolder[A <: Product : TypeTag](rdd: RDD[A]): DataFrameHolder = {
DataFrameHolder(_sqlContext.createDataFrame(rdd))
}

/**
* Creates a DataFrame from a local Seq of Product.
* @since 1.3.0
*/
implicit def localSeqToDataFrameHolder[A <: Product : TypeTag](data: Seq[A]): DataFrameHolder =
{
DataFrameHolder(_sqlContext.createDataFrame(data))
}

// Do NOT add more implicit conversions. They are likely to break source compatibility by
// making existing implicit conversions ambiguous. In particular, RDD[Double] is dangerous
// because of [[DoubleRDDFunctions]].

/**
* Creates a single column DataFrame from an RDD[Int].
* @since 1.3.0
*/
implicit def intRddToDataFrameHolder(data: RDD[Int]): DataFrameHolder = {
val dataType = IntegerType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.setInt(0, v)
row: InternalRow
}
}
DataFrameHolder(
_sqlContext.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}

/**
* Creates a single column DataFrame from an RDD[Long].
* @since 1.3.0
*/
implicit def longRddToDataFrameHolder(data: RDD[Long]): DataFrameHolder = {
val dataType = LongType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.setLong(0, v)
row: InternalRow
}
}
DataFrameHolder(
_sqlContext.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}

/**
* Creates a single column DataFrame from an RDD[String].
* @since 1.3.0
*/
implicit def stringRddToDataFrameHolder(data: RDD[String]): DataFrameHolder = {
val dataType = StringType
val rows = data.mapPartitions { iter =>
val row = new SpecificMutableRow(dataType :: Nil)
iter.map { v =>
row.update(0, UTF8String.fromString(v))
row: InternalRow
}
}
DataFrameHolder(
_sqlContext.internalCreateDataFrame(rows, StructType(StructField("_1", dataType) :: Nil)))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,14 @@
import org.junit.Before;
import org.junit.Test;

import org.apache.spark.SparkContext;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.sql.DataFrame;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.RowFactory;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.sql.test.TestSQLContext$;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
Expand All @@ -48,14 +48,16 @@ public class JavaApplySchemaSuite implements Serializable {

@Before
public void setUp() {
sqlContext = TestSQLContext$.MODULE$;
javaCtx = new JavaSparkContext(sqlContext.sparkContext());
SparkContext context = new SparkContext("local[*]", "testing");
javaCtx = new JavaSparkContext(context);
sqlContext = new SQLContext(context);
}

@After
public void tearDown() {
javaCtx = null;
sqlContext.sparkContext().stop();
sqlContext = null;
javaCtx = null;
}

public static class Person implements Serializable {
Expand Down
Loading