|
17 | 17 |
|
18 | 18 | package org.apache.spark.sql |
19 | 19 |
|
| 20 | +import org.apache.spark.{SparkContext, SparkFunSuite} |
| 21 | + |
20 | 22 | /** |
21 | 23 | * Test cases for the builder pattern of [[SparkSession]]. |
22 | 24 | */ |
23 | | -class SparkSessionBuilderSuite { |
| 25 | +class SparkSessionBuilderSuite extends SparkFunSuite { |
| 26 | + |
| 27 | + private var initialSession: SparkSession = _ |
| 28 | + |
| 29 | + private lazy val sparkContext: SparkContext = { |
| 30 | + initialSession = SparkSession.builder() |
| 31 | + .master("local") |
| 32 | + .config("spark.ui.enabled", value = false) |
| 33 | + .config("some-config", "v2") |
| 34 | + .getOrCreate() |
| 35 | + initialSession.sparkContext |
| 36 | + } |
| 37 | + |
| 38 | + test("create with config options and propagate them to SparkContext and SparkSession") { |
| 39 | + // Creating a new session with config - this works by just calling the lazy val |
| 40 | + sparkContext |
| 41 | + assert(initialSession.sparkContext.conf.get("some-config") == "v2") |
| 42 | + assert(initialSession.conf.get("some-config") == "v2") |
| 43 | + SparkSession.clearDefaultSession() |
| 44 | + } |
| 45 | + |
| 46 | + test("use global default session") { |
| 47 | + val session = SparkSession.builder().getOrCreate() |
| 48 | + assert(SparkSession.builder().getOrCreate() == session) |
| 49 | + SparkSession.clearDefaultSession() |
| 50 | + } |
| 51 | + |
| 52 | + test("config options are propagated to existing SparkSession") { |
| 53 | + val session1 = SparkSession.builder().config("spark-config1", "a").getOrCreate() |
| 54 | + assert(session1.conf.get("spark-config1") == "a") |
| 55 | + val session2 = SparkSession.builder().config("spark-config1", "b").getOrCreate() |
| 56 | + assert(session1 == session2) |
| 57 | + assert(session1.conf.get("spark-config1") == "b") |
| 58 | + SparkSession.clearDefaultSession() |
| 59 | + } |
| 60 | + |
| 61 | + test("use session from active thread session and propagate config options") { |
| 62 | + val defaultSession = SparkSession.builder().getOrCreate() |
| 63 | + val activeSession = defaultSession.newSession() |
| 64 | + SparkSession.setActiveSession(activeSession) |
| 65 | + val session = SparkSession.builder().config("spark-config2", "a").getOrCreate() |
| 66 | + |
| 67 | + assert(activeSession != defaultSession) |
| 68 | + assert(session == activeSession) |
| 69 | + assert(session.conf.get("spark-config2") == "a") |
| 70 | + SparkSession.clearActiveSession() |
| 71 | + |
| 72 | + assert(SparkSession.builder().getOrCreate() == defaultSession) |
| 73 | + SparkSession.clearDefaultSession() |
| 74 | + } |
| 75 | + |
| 76 | + test("create a new session if the default session has been stopped") { |
| 77 | + val defaultSession = SparkSession.builder().getOrCreate() |
| 78 | + SparkSession.setDefaultSession(defaultSession) |
| 79 | + defaultSession.stop() |
| 80 | + val newSession = SparkSession.builder().master("local").getOrCreate() |
| 81 | + assert(newSession != defaultSession) |
| 82 | + newSession.stop() |
| 83 | + } |
24 | 84 |
|
| 85 | + test("create a new session if the active thread session has been stopped") { |
| 86 | + val activeSession = SparkSession.builder().master("local").getOrCreate() |
| 87 | + SparkSession.setActiveSession(activeSession) |
| 88 | + activeSession.stop() |
| 89 | + val newSession = SparkSession.builder().master("local").getOrCreate() |
| 90 | + assert(newSession != activeSession) |
| 91 | + newSession.stop() |
| 92 | + } |
25 | 93 | } |
0 commit comments