@@ -23,6 +23,7 @@ import org.scalatest.BeforeAndAfterEach
2323import org .scalatest .time .SpanSugar ._
2424
2525import org .apache .spark .SparkSQLException
26+ import org .apache .spark .sql .SparkSession
2627import org .apache .spark .sql .pipelines .graph .{DataflowGraph , PipelineUpdateContextImpl }
2728import org .apache .spark .sql .pipelines .logging .PipelineEvent
2829import org .apache .spark .sql .test .SharedSparkSession
@@ -32,6 +33,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
3233 override def beforeEach (): Unit = {
3334 super .beforeEach()
3435 SparkConnectService .sessionManager.invalidateAllSessions()
36+ SparkConnectService .sessionManager.initializeBaseSession(spark.sparkContext)
3537 }
3638
3739 test(" sessionId needs to be an UUID" ) {
@@ -171,4 +173,51 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
171173 sessionHolder.getPipelineExecution(graphId).isEmpty,
172174 " pipeline execution was not removed" )
173175 }
176+
177+ test(" baseSession allows creating sessions after default session is cleared" ) {
178+ // Create a new session manager to test initialization
179+ val sessionManager = new SparkConnectSessionManager ()
180+
181+ // Initialize the base session with the test SparkContext
182+ sessionManager.initializeBaseSession(spark.sparkContext)
183+
184+ // Clear the default and active sessions to simulate the scenario where
185+ // SparkSession.active or SparkSession.getDefaultSession would fail
186+ SparkSession .clearDefaultSession()
187+ SparkSession .clearActiveSession()
188+
189+ // Create an isolated session - this should still work because we have baseSession
190+ val key = SessionKey (" user" , UUID .randomUUID().toString)
191+ val sessionHolder = sessionManager.getOrCreateIsolatedSession(key, None )
192+
193+ // Verify the session was created successfully
194+ assert(sessionHolder != null )
195+ assert(sessionHolder.session != null )
196+
197+ // Clean up
198+ sessionManager.closeSession(key)
199+ }
200+
201+ test(" initializeBaseSession is idempotent" ) {
202+ // Create a new session manager to test initialization
203+ val sessionManager = new SparkConnectSessionManager ()
204+
205+ // Initialize the base session multiple times
206+ sessionManager.initializeBaseSession(spark.sparkContext)
207+ val key1 = SessionKey (" user1" , UUID .randomUUID().toString)
208+ val sessionHolder1 = sessionManager.getOrCreateIsolatedSession(key1, None )
209+ val baseSessionUUID1 = sessionHolder1.session.sessionUUID
210+
211+ // Initialize again - should not change the base session
212+ sessionManager.initializeBaseSession(spark.sparkContext)
213+ val key2 = SessionKey (" user2" , UUID .randomUUID().toString)
214+ val sessionHolder2 = sessionManager.getOrCreateIsolatedSession(key2, None )
215+
216+ // Both sessions should be isolated from each other
217+ assert(sessionHolder1.session.sessionUUID != sessionHolder2.session.sessionUUID)
218+
219+ // Clean up
220+ sessionManager.closeSession(key1)
221+ sessionManager.closeSession(key2)
222+ }
174223}
0 commit comments