@@ -42,14 +42,17 @@ import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListen
42
42
* @param experimentalMethods Interface to add custom planning strategies and optimizers.
43
43
* @param functionRegistry Internal catalog for managing functions registered by the user.
44
44
* @param udfRegistration Interface exposed to the user for registering user-defined functions.
45
- * @param catalog Internal catalog for managing table and database states.
45
+ * @param catalogBuilder a function to create an internal catalog for managing table and database
46
+ * states.
46
47
* @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
47
- * @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
48
- * @param optimizer Logical query plan optimizer.
48
+ * @param analyzerBuilder A function to create the logical query plan analyzer for resolving
49
+ * unresolved attributes and relations.
50
+ * @param optimizerBuilder a function to create the logical query plan optimizer.
49
51
* @param planner Planner that converts optimized logical plans to physical plans.
50
52
* @param streamingQueryManager Interface to start and stop streaming queries.
51
53
* @param listenerManager Interface to register custom [[QueryExecutionListener ]]s.
52
- * @param resourceLoader Session shared resource loader to load JARs, files, etc.
54
+ * @param resourceLoaderBuilder a function to create a session shared resource loader to load JARs,
55
+ * files, etc.
53
56
* @param createQueryExecution Function used to create QueryExecution objects.
54
57
* @param createClone Function used to create clones of the session state.
55
58
*/
@@ -59,17 +62,26 @@ private[sql] class SessionState(
59
62
val experimentalMethods : ExperimentalMethods ,
60
63
val functionRegistry : FunctionRegistry ,
61
64
val udfRegistration : UDFRegistration ,
62
- val catalog : SessionCatalog ,
65
+ catalogBuilder : () => SessionCatalog ,
63
66
val sqlParser : ParserInterface ,
64
- val analyzer : Analyzer ,
65
- val optimizer : Optimizer ,
67
+ analyzerBuilder : () => Analyzer ,
68
+ optimizerBuilder : () => Optimizer ,
66
69
val planner : SparkPlanner ,
67
70
val streamingQueryManager : StreamingQueryManager ,
68
71
val listenerManager : ExecutionListenerManager ,
69
- val resourceLoader : SessionResourceLoader ,
72
+ resourceLoaderBuilder : () => SessionResourceLoader ,
70
73
createQueryExecution : LogicalPlan => QueryExecution ,
71
74
createClone : (SparkSession , SessionState ) => SessionState ) {
72
75
76
+ // The following fields are lazy to avoid creating the Hive client when creating SessionState.
77
+ lazy val catalog : SessionCatalog = catalogBuilder()
78
+
79
+ lazy val analyzer : Analyzer = analyzerBuilder()
80
+
81
+ lazy val optimizer : Optimizer = optimizerBuilder()
82
+
83
+ lazy val resourceLoader : SessionResourceLoader = resourceLoaderBuilder()
84
+
73
85
def newHadoopConf (): Configuration = SessionState .newHadoopConf(
74
86
sharedState.sparkContext.hadoopConfiguration,
75
87
conf)
0 commit comments