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
17 changes: 15 additions & 2 deletions core/src/test/scala/org/apache/spark/SharedSparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,23 @@ trait SharedSparkContext extends BeforeAndAfterAll with BeforeAndAfterEach { sel

var conf = new SparkConf(false)

/**
* Initialize the [[SparkContext]]. Generally, this is just called from beforeAll; however, in
* test using styles other than FunSuite, there is often code that relies on the session between
* test group constructs and the actual tests, which may need this session. It is purely a
* semantic difference, but semantically, it makes more sense to call 'initializeContext' between
* a 'describe' and an 'it' call than it does to call 'beforeAll'.
*/
protected def initializeContext(): Unit = {
if (null == _sc) {
_sc = new SparkContext(
"local[4]", "test", conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName))
}
}

override def beforeAll() {
super.beforeAll()
_sc = new SparkContext(
"local[4]", "test", conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName))
initializeContext()
}

override def afterAll() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@

package org.apache.spark.sql.catalyst.plans

import org.scalatest.Suite

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer
Expand All @@ -29,7 +31,13 @@ import org.apache.spark.sql.internal.SQLConf
/**
* Provides helper methods for comparing plans.
*/
trait PlanTest extends SparkFunSuite with PredicateHelper {
trait PlanTest extends SparkFunSuite with PlanTestBase

/**
* Provides helper methods for comparing plans, but without the overhead of
* mandating a FunSuite.
*/
trait PlanTestBase extends PredicateHelper { self: Suite =>

// TODO(gatorsmile): remove this from PlanTest and all the analyzer rules
protected def conf = SQLConf.get
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* 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.test

import org.scalatest.FlatSpec

/**
* The purpose of this suite is to make sure that generic FlatSpec-based scala
* tests work with a shared spark session
*/
class GenericFlatSpecSuite extends FlatSpec with SharedSparkSession {
Copy link
Member

Choose a reason for hiding this comment

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

These are testing tests? I get it, but I don't know if it's necessary. If FlatSpec didn't work, scalatest would hopefully catch it. If initializeSession / SharedSparkSession didn't, presumably tons of tests wouldn't work.

Copy link
Contributor Author

@nkronenfeld nkronenfeld Oct 23, 2017

Choose a reason for hiding this comment

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

Yeah, I wasn't totally sure about that either. I started off with the tests in another branch I wasn't going to include in the PR, just to make sure I'd done everything correctly.

I eventually came to the conclusion that they are useful. I think of it more in terms of preventing regressions; they should prevent a FunSuite dependecy creeping back in to SharedSparkSession.

For similar reasons, I should probably put similar tests in for SharedSparkContext.

import testImplicits._
initializeSession()
val ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS

"A Simple Dataset" should "have the specified number of elements" in {
assert(8 === ds.count)
}
it should "have the specified number of unique elements" in {
assert(8 === ds.distinct.count)
}
it should "have the specified number of elements in each column" in {
assert(8 === ds.select("_1").count)
assert(8 === ds.select("_2").count)
}
it should "have the correct number of distinct elements in each column" in {
assert(8 === ds.select("_1").distinct.count)
assert(4 === ds.select("_2").distinct.count)
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,47 @@
/*
* 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.test

import org.scalatest.FunSpec

/**
* The purpose of this suite is to make sure that generic FunSpec-based scala
* tests work with a shared spark session
*/
class GenericFunSpecSuite extends FunSpec with SharedSparkSession {
import testImplicits._
initializeSession()
val ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS

describe("Simple Dataset") {
it("should have the specified number of elements") {
assert(8 === ds.count)
}
it("should have the specified number of unique elements") {
assert(8 === ds.distinct.count)
}
it("should have the specified number of elements in each column") {
assert(8 === ds.select("_1").count)
assert(8 === ds.select("_2").count)
}
it("should have the correct number of distinct elements in each column") {
assert(8 === ds.select("_1").distinct.count)
assert(4 === ds.select("_2").distinct.count)
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.test

import org.scalatest.WordSpec

/**
* The purpose of this suite is to make sure that generic WordSpec-based scala
* tests work with a shared spark session
*/
class GenericWordSpecSuite extends WordSpec with SharedSparkSession {
import testImplicits._
initializeSession()
val ds = Seq((1, 1), (2, 1), (3, 2), (4, 2), (5, 3), (6, 3), (7, 4), (8, 4)).toDS

"A Simple Dataset" when {
"looked at as complete rows" should {
"have the specified number of elements" in {
assert(8 === ds.count)
}
"have the specified number of unique elements" in {
assert(8 === ds.distinct.count)
}
}
"refined to specific columns" should {
"have the specified number of elements in each column" in {
assert(8 === ds.select("_1").count)
assert(8 === ds.select("_2").count)
}
"have the correct number of distinct elements in each column" in {
assert(8 === ds.select("_1").distinct.count)
assert(4 === ds.select("_2").distinct.count)
}
}
}
}
Loading