[SPARK-28744][SQL][TEST] rename SharedSQLContext to SharedSparkSession
## What changes were proposed in this pull request?
The Spark SQL test framework needs to support 2 kinds of tests:
1. tests inside Spark to test Spark itself (extends `SparkFunSuite`)
2. test outside of Spark to test Spark applications (introduced at b57ed2245c
)
The class hierarchy of the major testing traits:
![image](https://user-images.githubusercontent.com/3182036/63088526-c0f0af80-bf87-11e9-9bed-c144c2486da9.png)
`PlanTestBase`, `SQLTestUtilsBase` and `SharedSparkSession` intentionally don't extend `SparkFunSuite`, so that they can be used for tests outside of Spark. Tests in Spark should extends `QueryTest` and/or `SharedSQLContext` in most cases.
However, the name is a little confusing. As a result, some test suites extend `SharedSparkSession` instead of `SharedSQLContext`. `SharedSparkSession` doesn't work well with `SparkFunSuite` as it doesn't have the special handling of thread auditing in `SharedSQLContext`. For example, you will see a warning starting with `===== POSSIBLE THREAD LEAK IN SUITE` when you run `DataFrameSelfJoinSuite`.
This PR proposes to rename `SharedSparkSession` to `SharedSparkSessionBase`, and rename `SharedSQLContext` to `SharedSparkSession`.
## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review https://spark.apache.org/contributing.html before opening a pull request.
Closes #25463 from cloud-fan/minor.
Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
parent
f999e00e9f
commit
97dc4c0bfc
|
@ -24,11 +24,11 @@ import org.apache.spark.sql.{RandomDataGenerator, Row}
|
|||
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal}
|
||||
import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class AvroCatalystDataConversionSuite extends SparkFunSuite
|
||||
with SharedSQLContext
|
||||
with SharedSparkSession
|
||||
with ExpressionEvalHelper {
|
||||
|
||||
private def roundTripTest(data: Literal): Unit = {
|
||||
|
|
|
@ -30,9 +30,9 @@ import org.apache.spark.sql.{QueryTest, Row}
|
|||
import org.apache.spark.sql.execution.LocalTableScanExec
|
||||
import org.apache.spark.sql.functions.{col, struct}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
|
||||
class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("roundtrip in to_avro and from_avro - int and string") {
|
||||
|
|
|
@ -28,10 +28,10 @@ import org.apache.spark.{SparkConf, SparkException}
|
|||
import org.apache.spark.sql.{QueryTest, Row}
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{StructField, StructType, TimestampType}
|
||||
|
||||
abstract class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
|
||||
abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
val dateSchema = s"""
|
||||
|
|
|
@ -38,11 +38,11 @@ import org.apache.spark.sql._
|
|||
import org.apache.spark.sql.TestingUDT.{IntervalData, NullData, NullUDT}
|
||||
import org.apache.spark.sql.execution.datasources.DataSource
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils {
|
||||
abstract class AvroSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
val episodesAvro = testFile("episodes.avro")
|
||||
|
|
|
@ -26,12 +26,10 @@ import scala.util.control.NonFatal
|
|||
import com.spotify.docker.client._
|
||||
import com.spotify.docker.client.exceptions.ImageNotFoundException
|
||||
import com.spotify.docker.client.messages.{ContainerConfig, HostConfig, PortBinding}
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
import org.scalatest.concurrent.Eventually
|
||||
import org.scalatest.time.SpanSugar._
|
||||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.util.DockerUtils
|
||||
|
||||
abstract class DatabaseOnDocker {
|
||||
|
@ -66,11 +64,7 @@ abstract class DatabaseOnDocker {
|
|||
def getStartupProcessName: Option[String]
|
||||
}
|
||||
|
||||
abstract class DockerJDBCIntegrationSuite
|
||||
extends SparkFunSuite
|
||||
with BeforeAndAfterAll
|
||||
with Eventually
|
||||
with SharedSQLContext {
|
||||
abstract class DockerJDBCIntegrationSuite extends SharedSparkSession with Eventually {
|
||||
|
||||
val db: DatabaseOnDocker
|
||||
|
||||
|
|
|
@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.{RowDataSourceScanExec, WholeStageCodegenE
|
|||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCPartition, JDBCRelation}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.tags.DockerTest
|
||||
|
||||
|
@ -54,7 +54,7 @@ import org.apache.spark.tags.DockerTest
|
|||
* repository.
|
||||
*/
|
||||
@DockerTest
|
||||
class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLContext {
|
||||
class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
override val db = new DatabaseOnDocker {
|
||||
|
|
|
@ -24,9 +24,9 @@ import org.apache.kafka.clients.producer.KafkaProducer
|
|||
import org.apache.kafka.common.serialization.ByteArraySerializer
|
||||
import org.scalatest.PrivateMethodTester
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class CachedKafkaProducerSuite extends SharedSQLContext with PrivateMethodTester with KafkaTest {
|
||||
class CachedKafkaProducerSuite extends SharedSparkSession with PrivateMethodTester with KafkaTest {
|
||||
|
||||
type KP = KafkaProducer[Array[Byte], Array[Byte]]
|
||||
|
||||
|
|
|
@ -30,10 +30,10 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer
|
|||
import org.scalatest.PrivateMethodTester
|
||||
|
||||
import org.apache.spark.{TaskContext, TaskContextImpl}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.util.ThreadUtils
|
||||
|
||||
class KafkaDataConsumerSuite extends SharedSQLContext with PrivateMethodTester {
|
||||
class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester {
|
||||
|
||||
protected var testUtils: KafkaTestUtils = _
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.spark.SparkContext
|
|||
import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.streaming.{StreamTest, Trigger}
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, TestSparkSession}
|
||||
|
||||
/**
|
||||
* This is a basic test trait which will set up a Kafka cluster that keeps only several records in
|
||||
|
@ -43,7 +43,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession}
|
|||
* does see missing offsets, you can check the earliest offset in `eventually` and make sure it's
|
||||
* not 0 rather than sleeping a hard-code duration.
|
||||
*/
|
||||
trait KafkaMissingOffsetsTest extends SharedSQLContext {
|
||||
trait KafkaMissingOffsetsTest extends SharedSparkSession {
|
||||
|
||||
protected var testUtils: KafkaTestUtils = _
|
||||
|
||||
|
|
|
@ -45,10 +45,10 @@ import org.apache.spark.sql.kafka010.KafkaSourceProvider._
|
|||
import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream
|
||||
import org.apache.spark.sql.streaming.{StreamTest, Trigger}
|
||||
import org.apache.spark.sql.streaming.util.StreamManualClock
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.util.CaseInsensitiveStringMap
|
||||
|
||||
abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest {
|
||||
abstract class KafkaSourceTest extends StreamTest with SharedSparkSession with KafkaTest {
|
||||
|
||||
protected var testUtils: KafkaTestUtils = _
|
||||
|
||||
|
|
|
@ -32,10 +32,10 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
|
|||
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources.BaseRelation
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
abstract class KafkaRelationSuiteBase extends QueryTest with SharedSQLContext with KafkaTest {
|
||||
abstract class KafkaRelationSuiteBase extends QueryTest with SharedSparkSession with KafkaTest {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.streaming.MemoryStream
|
|||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.streaming._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{BinaryType, DataType}
|
||||
|
||||
abstract class KafkaSinkSuiteBase extends QueryTest with SharedSQLContext with KafkaTest {
|
||||
abstract class KafkaSinkSuiteBase extends QueryTest with SharedSparkSession with KafkaTest {
|
||||
protected var testUtils: KafkaTestUtils = _
|
||||
|
||||
override def beforeAll(): Unit = {
|
||||
|
|
|
@ -21,9 +21,9 @@ import java.io.File
|
|||
|
||||
import org.apache.spark.sql.execution.streaming._
|
||||
import org.apache.spark.sql.streaming.OffsetSuite
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class KafkaSourceOffsetSuite extends OffsetSuite with SharedSQLContext {
|
||||
class KafkaSourceOffsetSuite extends OffsetSuite with SharedSparkSession {
|
||||
|
||||
compare(
|
||||
one = KafkaSourceOffset(("t", 0, 1L)),
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, Literal}
|
|||
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproxCountDistinctForIntervals
|
||||
import org.apache.spark.sql.catalyst.plans.logical.Aggregate
|
||||
import org.apache.spark.sql.execution.QueryExecution
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class ApproxCountDistinctForIntervalsQuerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
// ApproxCountDistinctForIntervals is used in equi-height histogram generation. An equi-height
|
||||
|
|
|
@ -23,12 +23,12 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
|
|||
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.DEFAULT_PERCENTILE_ACCURACY
|
||||
import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile.PercentileDigest
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* End-to-end tests for approximate percentile aggregate function.
|
||||
*/
|
||||
class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class ApproximatePercentileQuerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private val table = "percentile_test"
|
||||
|
|
|
@ -17,16 +17,14 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
|
||||
import org.apache.spark.internal.config.Tests.IS_TESTING
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator}
|
||||
import org.apache.spark.sql.catalyst.rules.RuleExecutor
|
||||
import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
abstract class BenchmarkQueryTest extends QueryTest with SharedSQLContext with BeforeAndAfterAll {
|
||||
abstract class BenchmarkQueryTest extends QueryTest with SharedSparkSession {
|
||||
|
||||
// When Utils.isTesting is true, the RuleExecutor will issue an exception when hitting
|
||||
// the max iteration of analyzer/optimizer batches.
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.spark.sql.execution.columnar._
|
|||
import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.storage.{RDDBlockId, StorageLevel}
|
||||
import org.apache.spark.storage.StorageLevel.{MEMORY_AND_DISK_2, MEMORY_ONLY}
|
||||
|
@ -40,7 +40,7 @@ import org.apache.spark.util.{AccumulatorContext, Utils}
|
|||
|
||||
private case class BigData(s: String)
|
||||
|
||||
class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext {
|
||||
class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -30,10 +30,10 @@ import org.apache.spark.sql.catalyst.expressions.NamedExpression
|
|||
import org.apache.spark.sql.execution.ProjectExec
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class ColumnExpressionSuite extends QueryTest with SharedSQLContext {
|
||||
class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private lazy val booleanData = {
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.sql.catalyst.expressions.CreateNamedStruct
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class ComplexTypesSuite extends QueryTest with SharedSQLContext {
|
||||
class ComplexTypesSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
override def beforeAll() {
|
||||
super.beforeAll()
|
||||
|
|
|
@ -20,10 +20,10 @@ package org.apache.spark.sql
|
|||
import org.apache.commons.math3.stat.inference.ChiSquareTest
|
||||
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
|
||||
class ConfigBehaviorSuite extends QueryTest with SharedSQLContext {
|
||||
class ConfigBehaviorSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -17,13 +17,13 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.util.sketch.CountMinSketch
|
||||
|
||||
/**
|
||||
* End-to-end test suite for count_min_sketch.
|
||||
*/
|
||||
class CountMinSketchAggQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class CountMinSketchAggQuerySuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
test("count-min sketch") {
|
||||
import testImplicits._
|
||||
|
|
|
@ -25,10 +25,10 @@ import scala.collection.JavaConverters._
|
|||
import org.apache.spark.SparkException
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class CsvFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class CsvFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("from_csv with empty options") {
|
||||
|
|
|
@ -27,13 +27,13 @@ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
|
|||
import org.apache.spark.sql.expressions.Window
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.test.SQLTestData.DecimalData
|
||||
import org.apache.spark.sql.types.DecimalType
|
||||
|
||||
case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double)
|
||||
|
||||
class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameAggregateSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
val absTol = 1e-8
|
||||
|
|
|
@ -19,12 +19,12 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.sql.catalyst.DefinedByConstructorParams
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* A test suite to test DataFrame/SQL functionalities with complex types (i.e. array, struct, map).
|
||||
*/
|
||||
class DataFrameComplexTypeSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("UDF on struct") {
|
||||
|
|
|
@ -30,13 +30,13 @@ import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation
|
|||
import org.apache.spark.sql.catalyst.util.DateTimeTestUtils
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
* Test suite for functions in [[org.apache.spark.sql.functions]].
|
||||
*/
|
||||
class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("array with column name") {
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.sql.catalyst.analysis.AnalysisTest
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DataFrameHintSuite extends AnalysisTest with SharedSQLContext {
|
||||
class DataFrameHintSuite extends AnalysisTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
lazy val df = spark.range(10)
|
||||
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DataFrameImplicitsSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameImplicitsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("RDD of tuples") {
|
||||
|
|
|
@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.plans.logical.Join
|
|||
import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DataFrameJoinSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameJoinSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("join - join using") {
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.spark.sql
|
|||
import scala.collection.JavaConverters._
|
||||
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DataFrameNaFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
def createDF(): DataFrame = {
|
||||
|
|
|
@ -22,10 +22,10 @@ import java.util.Locale
|
|||
import org.apache.spark.sql.catalyst.expressions.aggregate.PivotFirst
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class DataFramePivotSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFramePivotSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("pivot courses") {
|
||||
|
|
|
@ -27,10 +27,10 @@ import org.apache.spark.SparkException
|
|||
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
|
||||
class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventually {
|
||||
class DataFrameRangeSuite extends QueryTest with SharedSparkSession with Eventually {
|
||||
|
||||
test("SPARK-7150 range api") {
|
||||
// numSlice is greater than length
|
||||
|
|
|
@ -22,11 +22,11 @@ import java.sql.{Date, Timestamp}
|
|||
import org.apache.spark.sql.catalyst.plans.logical.Union
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
|
||||
import org.apache.spark.sql.test.SQLTestData.NullStrings
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class DataFrameSetOperationsSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameSetOperationsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("except") {
|
||||
|
|
|
@ -25,10 +25,10 @@ import org.apache.spark.internal.Logging
|
|||
import org.apache.spark.sql.execution.stat.StatFunctions
|
||||
import org.apache.spark.sql.functions.{col, lit, struct}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
|
||||
|
||||
class DataFrameStatSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameStatSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private def toLetter(i: Int): String = (i + 97).toChar.toString
|
||||
|
@ -449,7 +449,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext {
|
|||
}
|
||||
|
||||
|
||||
class DataFrameStatPerfSuite extends QueryTest with SharedSQLContext with Logging {
|
||||
class DataFrameStatPerfSuite extends QueryTest with SharedSparkSession with Logging {
|
||||
|
||||
// Turn on this test if you want to test the performance of approximate quantiles.
|
||||
ignore("computing quantiles should not take much longer than describe()") {
|
||||
|
|
|
@ -37,13 +37,13 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec
|
|||
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
|
||||
import org.apache.spark.sql.test.SQLTestData.{NullStrings, TestData2}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.util.Utils
|
||||
import org.apache.spark.util.random.XORShiftRandom
|
||||
|
||||
class DataFrameSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("analysis error should be eagerly reported") {
|
||||
|
|
|
@ -21,10 +21,10 @@ import org.scalatest.BeforeAndAfterEach
|
|||
|
||||
import org.apache.spark.sql.catalyst.plans.logical.Expand
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.StringType
|
||||
|
||||
class DataFrameTimeWindowingSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach {
|
||||
class DataFrameTimeWindowingSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -18,7 +18,7 @@
|
|||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
|
@ -27,7 +27,7 @@ import org.apache.spark.sql.types._
|
|||
* This is here for now so I can make sure Tungsten project is tested without refactoring existing
|
||||
* end-to-end test infra. In the long run this should just go away.
|
||||
*/
|
||||
class DataFrameTungstenSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameTungstenSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("test simple types") {
|
||||
|
|
|
@ -21,12 +21,12 @@ import java.sql.Date
|
|||
|
||||
import org.apache.spark.sql.expressions.Window
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* Window frame testing for DataFrame API.
|
||||
*/
|
||||
class DataFrameWindowFramesSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameWindowFramesSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("lead/lag with empty data frame") {
|
||||
|
|
|
@ -25,13 +25,13 @@ import org.apache.spark.sql.execution.exchange.Exchange
|
|||
import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
* Window function testing for DataFrame API.
|
||||
*/
|
||||
class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class DataFrameWindowFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
|
|||
import org.apache.spark.sql.expressions.Aggregator
|
||||
import org.apache.spark.sql.expressions.scalalang.typed
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType}
|
||||
|
||||
|
||||
|
@ -221,7 +221,7 @@ case class OptionBooleanIntAggregator(colName: String)
|
|||
def OptionalBoolIntEncoder: Encoder[Option[(Boolean, Int)]] = ExpressionEncoder()
|
||||
}
|
||||
|
||||
class DatasetAggregatorSuite extends QueryTest with SharedSQLContext {
|
||||
class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b)
|
||||
|
|
|
@ -22,11 +22,11 @@ import org.scalatest.time.SpanSugar._
|
|||
|
||||
import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.storage.StorageLevel
|
||||
|
||||
|
||||
class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits {
|
||||
class DatasetCacheSuite extends QueryTest with SharedSparkSession with TimeLimits {
|
||||
import testImplicits._
|
||||
|
||||
/**
|
||||
|
|
|
@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.expressions.objects.ExternalMapToCatalyst
|
|||
import org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
|
||||
import org.apache.spark.sql.functions.expr
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DatasetOptimizationSuite extends QueryTest with SharedSQLContext {
|
||||
class DatasetOptimizationSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("SPARK-26619: Prune the unused serializers from SerializeFromObject") {
|
||||
|
|
|
@ -22,7 +22,7 @@ import scala.collection.immutable.Queue
|
|||
import scala.collection.mutable.{LinkedHashMap => LHMap}
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
case class IntClass(value: Int)
|
||||
|
||||
|
@ -47,7 +47,7 @@ package object packageobject {
|
|||
case class PackageClass(value: Int)
|
||||
}
|
||||
|
||||
class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext {
|
||||
class DatasetPrimitiveSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("toDS") {
|
||||
|
|
|
@ -23,12 +23,12 @@ import com.esotericsoftware.kryo.io.{Input, Output}
|
|||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.internal.config.Kryo._
|
||||
import org.apache.spark.serializer.KryoRegistrator
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* Test suite to test Kryo custom registrators.
|
||||
*/
|
||||
class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext {
|
||||
class DatasetSerializerRegistratorSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.spark.sql.execution.streaming.MemoryStream
|
|||
import org.apache.spark.sql.expressions.UserDefinedFunction
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2)
|
||||
|
@ -49,7 +49,7 @@ object TestForTypeAlias {
|
|||
def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2))
|
||||
}
|
||||
|
||||
class DatasetSuite extends QueryTest with SharedSQLContext {
|
||||
class DatasetSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private implicit val ordering = Ordering.by((c: ClassData) => c.a -> c.b)
|
||||
|
|
|
@ -26,10 +26,10 @@ import java.util.concurrent.TimeUnit
|
|||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.unsafe.types.CalendarInterval
|
||||
|
||||
class DateFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class DateFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("function current_date") {
|
||||
|
|
|
@ -18,10 +18,10 @@
|
|||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
class ExplainSuite extends QueryTest with SharedSQLContext {
|
||||
class ExplainSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
/**
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow
|
|||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
|
||||
import org.apache.spark.sql.execution.SparkPlan
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
case class FastOperator(output: Seq[Attribute]) extends SparkPlan {
|
||||
|
||||
|
@ -46,7 +46,7 @@ object TestStrategy extends Strategy {
|
|||
}
|
||||
}
|
||||
|
||||
class ExtraStrategiesSuite extends QueryTest with SharedSQLContext {
|
||||
class ExtraStrategiesSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("insert an extraStrategy") {
|
||||
|
|
|
@ -24,7 +24,6 @@ import java.util.Locale
|
|||
import scala.collection.mutable
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
|
||||
import org.apache.spark.SparkException
|
||||
import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd}
|
||||
|
@ -35,11 +34,11 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable
|
|||
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
|
||||
class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with BeforeAndAfterAll {
|
||||
class FileBasedDataSourceSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
override def beforeAll(): Unit = {
|
||||
|
|
|
@ -22,10 +22,10 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Generator}
|
|||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{IntegerType, StructType}
|
||||
|
||||
class GeneratorFunctionSuite extends QueryTest with SharedSQLContext {
|
||||
class GeneratorFunctionSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("stack") {
|
||||
|
|
|
@ -28,9 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._
|
|||
import org.apache.spark.sql.catalyst.rules.RuleExecutor
|
||||
import org.apache.spark.sql.execution.joins._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class JoinHintSuite extends PlanTest with SharedSQLContext {
|
||||
class JoinHintSuite extends PlanTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
lazy val df = spark.range(10)
|
||||
|
|
|
@ -31,10 +31,10 @@ import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec}
|
|||
import org.apache.spark.sql.execution.joins._
|
||||
import org.apache.spark.sql.execution.python.BatchEvalPythonExec
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
class JoinSuite extends QueryTest with SharedSQLContext {
|
||||
class JoinSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -25,10 +25,10 @@ import collection.JavaConverters._
|
|||
import org.apache.spark.SparkException
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class JsonFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("function get_json_object") {
|
||||
|
|
|
@ -21,14 +21,14 @@ import java.nio.charset.StandardCharsets
|
|||
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.functions.{log => logarithm}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
private object MathFunctionsTestData {
|
||||
case class DoubleData(a: java.lang.Double, b: java.lang.Double)
|
||||
case class NullDoubles(a: java.lang.Double)
|
||||
}
|
||||
|
||||
class MathFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class MathFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import MathFunctionsTestData._
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -21,12 +21,12 @@ import java.io.File
|
|||
|
||||
import org.apache.spark.{SparkConf, SparkException}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* Test suite to handle metadata cache related.
|
||||
*/
|
||||
abstract class MetadataCacheSuite extends QueryTest with SharedSQLContext {
|
||||
abstract class MetadataCacheSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
/** Removes one data file in the given directory. */
|
||||
protected def deleteOneFileInDirectory(dir: File): Unit = {
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class MiscFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class MiscFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("reflect and java_method") {
|
||||
|
|
|
@ -416,7 +416,7 @@ object QueryTest {
|
|||
}
|
||||
}
|
||||
|
||||
class QueryTestSuite extends QueryTest with test.SharedSQLContext {
|
||||
class QueryTestSuite extends QueryTest with test.SharedSparkSession {
|
||||
test("SPARK-16940: checkAnswer should raise TestFailedException for wrong results") {
|
||||
intercept[org.scalatest.exceptions.TestFailedException] {
|
||||
checkAnswer(sql("SELECT 1"), Row(2) :: Nil)
|
||||
|
|
|
@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.expressions.{CaseWhen, If, Literal}
|
|||
import org.apache.spark.sql.execution.LocalTableScanExec
|
||||
import org.apache.spark.sql.functions.{lit, when}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.BooleanType
|
||||
|
||||
class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSQLContext {
|
||||
class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("SPARK-25860: Replace Literal(null, _) with FalseLiteral whenever possible") {
|
||||
|
|
|
@ -19,11 +19,11 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, SpecificInternalRow}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
class RowSuite extends SparkFunSuite with SharedSQLContext {
|
||||
class RowSuite extends SparkFunSuite with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("create row") {
|
||||
|
|
|
@ -33,11 +33,11 @@ import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan
|
|||
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, TestSQLContext}
|
||||
import org.apache.spark.sql.test.SQLTestData._
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class SQLQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class SQLQuerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile}
|
|||
import org.apache.spark.sql.execution.HiveResult.hiveResultString
|
||||
import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
/**
|
||||
|
@ -102,7 +102,7 @@ import org.apache.spark.sql.types.StructType
|
|||
* Therefore, UDF test cases should have single input and output files but executed by three
|
||||
* different types of UDFs. See 'udf/udf-inner-join.sql' as an example.
|
||||
*/
|
||||
class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
|
||||
class SQLQueryTestSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import IntegratedUDFTestUtils._
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.spark.sql
|
|||
import java.sql.{Date, Timestamp}
|
||||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
case class ReflectData(
|
||||
stringField: String,
|
||||
|
@ -74,7 +74,7 @@ case class ComplexReflectData(
|
|||
mapFieldContainsNull: Map[Int, Option[Long]],
|
||||
dataField: Data)
|
||||
|
||||
class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSQLContext {
|
||||
class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("query case class RDD") {
|
||||
|
|
|
@ -19,9 +19,9 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.{SparkConf, SparkFunSuite}
|
||||
import org.apache.spark.serializer.JavaSerializer
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class SerializationSuite extends SparkFunSuite with SharedSQLContext {
|
||||
class SerializationSuite extends SparkFunSuite with SharedSparkSession {
|
||||
|
||||
test("[SPARK-5235] SQLContext should be serializable") {
|
||||
val spark = SparkSession.builder.getOrCreate()
|
||||
|
|
|
@ -19,10 +19,10 @@ package org.apache.spark.sql
|
|||
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSQLContext
|
||||
class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSparkSession
|
||||
|
||||
abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils {
|
||||
import testImplicits._
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat
|
|||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.test.SQLTestData.ArrayData
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.util.Utils
|
||||
|
@ -39,7 +39,7 @@ import org.apache.spark.util.Utils
|
|||
/**
|
||||
* End-to-end suite testing statistics collection and use on both entire table and columns.
|
||||
*/
|
||||
class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSQLContext {
|
||||
class StatisticsCollectionSuite extends StatisticsCollectionTestBase with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("estimates the size of a limit 0 on outer join") {
|
||||
|
|
|
@ -18,10 +18,10 @@
|
|||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
|
||||
class StringFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class StringFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("string concat") {
|
||||
|
|
|
@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan, Sort}
|
|||
import org.apache.spark.sql.execution.{ColumnarToRowExec, ExecSubqueryExpression, FileSourceScanExec, InputAdapter, ReusedSubqueryExec, ScalarSubquery, SubqueryExec, WholeStageCodegenExec}
|
||||
import org.apache.spark.sql.execution.datasources.FileScanRDD
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class SubquerySuite extends QueryTest with SharedSQLContext {
|
||||
class SubquerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggreg
|
|||
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
|
||||
import org.apache.spark.sql.expressions.Window
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class TypedImperativeAggregateSuite extends QueryTest with SharedSQLContext {
|
||||
class TypedImperativeAggregateSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectComm
|
|||
import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand
|
||||
import org.apache.spark.sql.functions.{lit, udf}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.test.SQLTestData._
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.sql.util.QueryExecutionListener
|
||||
|
@ -36,7 +36,7 @@ import org.apache.spark.sql.util.QueryExecutionListener
|
|||
|
||||
private case class FunctionResult(f1: String, f2: String)
|
||||
|
||||
class UDFSuite extends QueryTest with SharedSQLContext {
|
||||
class UDFSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("built-in fixed arity expressions") {
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
|
|||
import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetTest
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseVector) {
|
||||
|
@ -101,7 +101,7 @@ private[spark] class ExampleSubTypeUDT extends UserDefinedType[IExampleSubType]
|
|||
override def userClass: Class[IExampleSubType] = classOf[IExampleSubType]
|
||||
}
|
||||
|
||||
class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetTest
|
||||
class UserDefinedTypeSuite extends QueryTest with SharedSparkSession with ParquetTest
|
||||
with ExpressionEvalHelper {
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
|
||||
package org.apache.spark.sql
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* End-to-end tests for xpath expressions.
|
||||
*/
|
||||
class XPathFunctionsSuite extends QueryTest with SharedSQLContext {
|
||||
class XPathFunctionsSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("xpath_boolean") {
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
|
||||
package org.apache.spark.sql.api.r
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class SQLUtilsSuite extends SharedSQLContext {
|
||||
class SQLUtilsSuite extends SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec
|
|||
import org.apache.spark.sql.execution.joins.HashedRelation
|
||||
import org.apache.spark.sql.functions.broadcast
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class BroadcastExchangeSuite extends SparkPlanTest with SharedSQLContext {
|
||||
class BroadcastExchangeSuite extends SparkPlanTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -23,12 +23,12 @@ import org.apache.spark.sql.{DataFrame, QueryTest}
|
|||
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
|
||||
import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
/**
|
||||
* Test suite base for testing the redaction of DataSourceScanExec/BatchScanExec.
|
||||
*/
|
||||
abstract class DataSourceScanRedactionTest extends QueryTest with SharedSQLContext {
|
||||
abstract class DataSourceScanRedactionTest extends QueryTest with SharedSparkSession {
|
||||
|
||||
override protected def sparkConf: SparkConf = super.sparkConf
|
||||
.set("spark.redaction.string.regex", "file:/[^\\]\\s]+")
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, IdentityB
|
|||
import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec}
|
||||
import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class ExchangeSuite extends SparkPlanTest with SharedSQLContext {
|
||||
class ExchangeSuite extends SparkPlanTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("shuffling UnsafeRows in exchange") {
|
||||
|
|
|
@ -21,10 +21,10 @@ import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
|
|||
import org.apache.spark.sql.catalog.Table
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
class GlobalTempViewSuite extends QueryTest with SharedSQLContext {
|
||||
class GlobalTempViewSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
override protected def beforeAll(): Unit = {
|
||||
|
|
|
@ -19,10 +19,9 @@ package org.apache.spark.sql.execution
|
|||
|
||||
import java.sql.{Date, Timestamp}
|
||||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext}
|
||||
import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession}
|
||||
|
||||
class HiveResultSuite extends SparkFunSuite with SharedSQLContext {
|
||||
class HiveResultSuite extends SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("date formatting in hive result") {
|
||||
|
|
|
@ -23,9 +23,9 @@ import org.apache.spark.sql._
|
|||
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_METADATA_ONLY
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
override def beforeAll(): Unit = {
|
||||
|
|
|
@ -29,10 +29,10 @@ import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchan
|
|||
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
class PlannerSuite extends SharedSQLContext {
|
||||
class PlannerSuite extends SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.plans.QueryPlan
|
|||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation}
|
||||
import org.apache.spark.sql.catalyst.trees.TreeNodeTag
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
case class QueryExecutionTestRecord(
|
||||
c0: Int, c1: Int, c2: Int, c3: Int, c4: Int,
|
||||
|
@ -33,7 +33,7 @@ case class QueryExecutionTestRecord(
|
|||
c20: Int, c21: Int, c22: Int, c23: Int, c24: Int,
|
||||
c25: Int, c26: Int)
|
||||
|
||||
class QueryExecutionSuite extends SharedSQLContext {
|
||||
class QueryExecutionSuite extends SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
def checkDumpedPlans(path: String, expected: Int): Unit = {
|
||||
|
|
|
@ -17,9 +17,9 @@
|
|||
|
||||
package org.apache.spark.sql.execution
|
||||
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext {
|
||||
class QueryPlanningTrackerEndToEndSuite extends SharedSparkSession {
|
||||
|
||||
test("programmatic API") {
|
||||
val df = spark.range(1000).selectExpr("count(*)")
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.spark.sql._
|
|||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
|
||||
import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
|
||||
|
||||
class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext
|
||||
class SimpleSQLViewSuite extends SQLViewSuite with SharedSparkSession
|
||||
|
||||
/**
|
||||
* A suite for testing view related functionality.
|
||||
|
|
|
@ -20,7 +20,7 @@ package org.apache.spark.sql.execution
|
|||
import org.apache.spark.TestUtils.assertSpilled
|
||||
import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
|
||||
import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
case class WindowData(month: Int, area: String, product: Int)
|
||||
|
||||
|
@ -28,7 +28,7 @@ case class WindowData(month: Int, area: String, product: Int)
|
|||
/**
|
||||
* Test suite for SQL window functions.
|
||||
*/
|
||||
class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext {
|
||||
class SQLWindowFunctionSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -23,13 +23,13 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project}
|
|||
import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.IntegerType
|
||||
|
||||
/**
|
||||
* Tests for the sameResult function for [[SparkPlan]]s.
|
||||
*/
|
||||
class SameResultSuite extends QueryTest with SharedSQLContext {
|
||||
class SameResultSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("FileSourceScanExec: different orders of data filters and partition filters") {
|
||||
|
|
|
@ -22,14 +22,14 @@ import scala.util.Random
|
|||
import org.apache.spark.AccumulatorSuite
|
||||
import org.apache.spark.sql.{RandomDataGenerator, Row}
|
||||
import org.apache.spark.sql.catalyst.dsl.expressions._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
/**
|
||||
* Test sorting. Many of the test cases generate random data and compares the sorted result with one
|
||||
* sorted by a reference implementation ([[ReferenceSort]]).
|
||||
*/
|
||||
class SortSuite extends SparkPlanTest with SharedSQLContext {
|
||||
class SortSuite extends SparkPlanTest with SharedSparkSession {
|
||||
import testImplicits.newProductEncoder
|
||||
import testImplicits.localSeqToDatasetHolder
|
||||
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.spark.SparkEnv
|
|||
import org.apache.spark.sql.QueryTest
|
||||
import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class SparkPlanSuite extends QueryTest with SharedSQLContext {
|
||||
class SparkPlanSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
test("SPARK-21619 execution of a canonicalized plan should fail") {
|
||||
val plan = spark.range(10).queryExecution.executedPlan.canonicalized
|
||||
|
|
|
@ -20,9 +20,9 @@ package org.apache.spark.sql.execution
|
|||
import org.apache.spark.sql.Strategy
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, ReturnAnswer, Union}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class SparkPlannerSuite extends SharedSQLContext {
|
||||
class SparkPlannerSuite extends SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
test("Ensure to go down only the first branch, not any other possible branches") {
|
||||
|
|
|
@ -22,11 +22,11 @@ import scala.util.Random
|
|||
import org.apache.spark.sql.{DataFrame, Row}
|
||||
import org.apache.spark.sql.catalyst.dsl.expressions._
|
||||
import org.apache.spark.sql.catalyst.expressions.Literal
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
|
||||
|
||||
class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext {
|
||||
class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSparkSession {
|
||||
|
||||
private var rand: Random = _
|
||||
private var seed: Long = 0
|
||||
|
|
|
@ -31,7 +31,7 @@ import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED
|
|||
import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.UnsafeRow
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
|
@ -43,7 +43,7 @@ import org.apache.spark.unsafe.types.UTF8String
|
|||
class UnsafeFixedWidthAggregationMapSuite
|
||||
extends SparkFunSuite
|
||||
with Matchers
|
||||
with SharedSQLContext {
|
||||
with SharedSparkSession {
|
||||
|
||||
import UnsafeFixedWidthAggregationMap._
|
||||
|
||||
|
|
|
@ -27,14 +27,14 @@ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager}
|
|||
import org.apache.spark.sql.{RandomDataGenerator, Row}
|
||||
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
|
||||
import org.apache.spark.sql.catalyst.expressions.{InterpretedOrdering, UnsafeProjection, UnsafeRow}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.unsafe.map.BytesToBytesMap
|
||||
|
||||
/**
|
||||
* Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data.
|
||||
*/
|
||||
class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSQLContext {
|
||||
class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSparkSession {
|
||||
private val keyTypes = Seq(IntegerType, FloatType, DoubleType, StringType)
|
||||
private val valueTypes = Seq(IntegerType, FloatType, DoubleType, StringType)
|
||||
|
||||
|
|
|
@ -26,10 +26,10 @@ import org.apache.spark.sql.execution.joins.SortMergeJoinExec
|
|||
import org.apache.spark.sql.expressions.scalalang.typed
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{IntegerType, StringType, StructType}
|
||||
|
||||
class WholeStageCodegenSuite extends QueryTest with SharedSQLContext {
|
||||
class WholeStageCodegenSuite extends QueryTest with SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -23,9 +23,9 @@ import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec
|
|||
import org.apache.spark.sql.execution.exchange.Exchange
|
||||
import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext {
|
||||
class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -27,20 +27,19 @@ import org.apache.arrow.memory.RootAllocator
|
|||
import org.apache.arrow.vector.{VectorLoader, VectorSchemaRoot}
|
||||
import org.apache.arrow.vector.ipc.JsonFileReader
|
||||
import org.apache.arrow.vector.util.{ByteArrayReadableSeekableByteChannel, Validator}
|
||||
import org.scalatest.BeforeAndAfterAll
|
||||
|
||||
import org.apache.spark.{SparkException, TaskContext}
|
||||
import org.apache.spark.sql.{DataFrame, Row}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.util.DateTimeUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{BinaryType, Decimal, IntegerType, StructField, StructType}
|
||||
import org.apache.spark.sql.util.ArrowUtils
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
|
||||
class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll {
|
||||
class ArrowConvertersSuite extends SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
private var tempDataPath: String = _
|
||||
|
|
|
@ -27,13 +27,13 @@ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
|
|||
import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.test.SQLTestData._
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.storage.StorageLevel._
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext {
|
||||
class InMemoryColumnarQuerySuite extends QueryTest with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
setupTestData()
|
||||
|
|
|
@ -17,19 +17,12 @@
|
|||
|
||||
package org.apache.spark.sql.execution.columnar
|
||||
|
||||
import org.scalatest.BeforeAndAfterEach
|
||||
|
||||
import org.apache.spark.SparkFunSuite
|
||||
import org.apache.spark.sql.DataFrame
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.test.SQLTestData._
|
||||
|
||||
|
||||
class PartitionBatchPruningSuite
|
||||
extends SparkFunSuite
|
||||
with BeforeAndAfterEach
|
||||
with SharedSQLContext {
|
||||
class PartitionBatchPruningSuite extends SharedSparkSession {
|
||||
|
||||
import testImplicits._
|
||||
|
||||
|
|
|
@ -35,10 +35,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, Log
|
|||
import org.apache.spark.sql.execution.SparkSqlParser
|
||||
import org.apache.spark.sql.execution.datasources.CreateTable
|
||||
import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
|
||||
|
||||
class DDLParserSuite extends AnalysisTest with SharedSQLContext {
|
||||
class DDLParserSuite extends AnalysisTest with SharedSparkSession {
|
||||
private lazy val parser = new SparkSqlParser(new SQLConf)
|
||||
|
||||
private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = {
|
||||
|
|
|
@ -33,12 +33,12 @@ import org.apache.spark.sql.catalyst.catalog._
|
|||
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
|
||||
import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils}
|
||||
import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
|
||||
import org.apache.spark.sql.types._
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
|
||||
class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with BeforeAndAfterEach {
|
||||
class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession {
|
||||
import testImplicits._
|
||||
|
||||
override def afterEach(): Unit = {
|
||||
|
|
|
@ -21,9 +21,9 @@ import org.apache.spark.sql.catalyst.dsl.expressions._
|
|||
import org.apache.spark.sql.catalyst.expressions._
|
||||
import org.apache.spark.sql.catalyst.plans.PlanTest
|
||||
import org.apache.spark.sql.sources
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class DataSourceStrategySuite extends PlanTest with SharedSQLContext {
|
||||
class DataSourceStrategySuite extends PlanTest with SharedSparkSession {
|
||||
|
||||
test("translate simple expression") {
|
||||
val attrInt = 'cint.int
|
||||
|
|
|
@ -19,11 +19,11 @@ package org.apache.spark.sql.execution.datasources
|
|||
|
||||
import org.apache.spark.sql.{QueryTest, Row}
|
||||
import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
|
||||
class FileFormatWriterSuite
|
||||
extends QueryTest
|
||||
with SharedSQLContext
|
||||
with SharedSparkSession
|
||||
with CodegenInterpretedPlanTest{
|
||||
|
||||
import testImplicits._
|
||||
|
|
|
@ -30,11 +30,11 @@ import org.apache.spark.sql.SparkSession
|
|||
import org.apache.spark.sql.catalyst.util._
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
|
||||
import org.apache.spark.util.KnownSizeEstimation
|
||||
|
||||
class FileIndexSuite extends SharedSQLContext {
|
||||
class FileIndexSuite extends SharedSparkSession {
|
||||
|
||||
private class TestInMemoryFileIndex(
|
||||
spark: SparkSession,
|
||||
|
|
|
@ -35,11 +35,11 @@ import org.apache.spark.sql.execution.{DataSourceScanExec, SparkPlan}
|
|||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.test.SharedSQLContext
|
||||
import org.apache.spark.sql.test.SharedSparkSession
|
||||
import org.apache.spark.sql.types.{IntegerType, StructType}
|
||||
import org.apache.spark.util.Utils
|
||||
|
||||
class FileSourceStrategySuite extends QueryTest with SharedSQLContext with PredicateHelper {
|
||||
class FileSourceStrategySuite extends QueryTest with SharedSparkSession with PredicateHelper {
|
||||
import testImplicits._
|
||||
|
||||
protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1")
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Reference in a new issue