Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
test fixes
  • Loading branch information
jose-torres committed Mar 28, 2018
commit 8e191250864e9a6fe36b5f705f2aedfb366b0afb
Original file line number Diff line number Diff line change
Expand Up @@ -39,16 +39,6 @@ class RateSourceSuite extends StreamTest {

import testImplicits._

protected override def beforeAll(): Unit = {
super.beforeAll()
SparkSession.setActiveSession(spark)
}

override def afterAll(): Unit = {
SparkSession.clearActiveSession()
super.afterAll()
}

case class AdvanceRateManualClock(seconds: Long) extends AddData {
override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = {
assert(query.nonEmpty)
Expand Down Expand Up @@ -257,19 +247,19 @@ class RateSourceSuite extends StreamTest {
.distinct()
testStream(input)(
AdvanceRateManualClock(2),
ExpectFailure[TreeNodeException[_]](t => {
ExpectFailure[ArithmeticException](t => {
Seq("overflow", "rowsPerSecond").foreach { msg =>
assert(t.getCause.getMessage.contains(msg))
assert(t.getMessage.contains(msg))
}
})
)
}

testQuietly("illegal option values") {
def testIllegalOptionValue(
option: String,
value: String,
expectedMessages: Seq[String]): Unit = {
option: String,
value: String,
expectedMessages: Seq[String]): Unit = {
val e = intercept[IllegalArgumentException] {
spark.readStream
.format("rate")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with
if (loadTestDataBeforeTests) {
loadTestData()
}
SparkSession.setActiveSession(spark)
Copy link
Member

Choose a reason for hiding this comment

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

The active session should be set before we execute the plan, right? For example, in QueryExecution for each query. What is the reason we need to do it here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The active session is required for instantiating the DataSourceReader, which is done at planning time (spark.readStream.{...}.load()) in order to determine the schema.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Discussed offline. What we should do instead is set the default session when the test spark session is initialized, since that initialization doesn't invoke SparkSession.getOrCreate() which normally sets it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

(This issue was spun off into #20926.)

}

/**
Expand Down