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
Original file line number Diff line number Diff line change
Expand Up @@ -287,14 +287,14 @@ abstract class BaseSessionStateBuilder(
experimentalMethods,
functionRegistry,
udfRegistration,
catalog,
() => catalog,
sqlParser,
analyzer,
optimizer,
() => analyzer,
() => optimizer,
planner,
streamingQueryManager,
listenerManager,
resourceLoader,
() => resourceLoader,
createQueryExecution,
createClone)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,14 +42,17 @@ import org.apache.spark.sql.util.{ExecutionListenerManager, QueryExecutionListen
* @param experimentalMethods Interface to add custom planning strategies and optimizers.
* @param functionRegistry Internal catalog for managing functions registered by the user.
* @param udfRegistration Interface exposed to the user for registering user-defined functions.
* @param catalog Internal catalog for managing table and database states.
* @param catalogBuilder a function to create an internal catalog for managing table and database
* states.
* @param sqlParser Parser that extracts expressions, plans, table identifiers etc. from SQL texts.
* @param analyzer Logical query plan analyzer for resolving unresolved attributes and relations.
* @param optimizer Logical query plan optimizer.
* @param analyzerBuilder A function to create the logical query plan analyzer for resolving
* unresolved attributes and relations.
* @param optimizerBuilder a function to create the logical query plan optimizer.
* @param planner Planner that converts optimized logical plans to physical plans.
* @param streamingQueryManager Interface to start and stop streaming queries.
* @param listenerManager Interface to register custom [[QueryExecutionListener]]s.
* @param resourceLoader Session shared resource loader to load JARs, files, etc.
* @param resourceLoaderBuilder a function to create a session shared resource loader to load JARs,
* files, etc.
* @param createQueryExecution Function used to create QueryExecution objects.
* @param createClone Function used to create clones of the session state.
*/
Expand All @@ -59,17 +62,26 @@ private[sql] class SessionState(
val experimentalMethods: ExperimentalMethods,
val functionRegistry: FunctionRegistry,
val udfRegistration: UDFRegistration,
val catalog: SessionCatalog,
catalogBuilder: () => SessionCatalog,
val sqlParser: ParserInterface,
val analyzer: Analyzer,
val optimizer: Optimizer,
analyzerBuilder: () => Analyzer,
optimizerBuilder: () => Optimizer,
val planner: SparkPlanner,
val streamingQueryManager: StreamingQueryManager,
val listenerManager: ExecutionListenerManager,
val resourceLoader: SessionResourceLoader,
resourceLoaderBuilder: () => SessionResourceLoader,
createQueryExecution: LogicalPlan => QueryExecution,
createClone: (SparkSession, SessionState) => SessionState) {

// The following fields are lazy to avoid creating the Hive client when creating SessionState.
lazy val catalog: SessionCatalog = catalogBuilder()

lazy val analyzer: Analyzer = analyzerBuilder()

lazy val optimizer: Optimizer = optimizerBuilder()

lazy val resourceLoader: SessionResourceLoader = resourceLoaderBuilder()

def newHadoopConf(): Configuration = SessionState.newHadoopConf(
sharedState.sparkContext.hadoopConfiguration,
conf)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
Copy link
Member Author

Choose a reason for hiding this comment

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

This test is in hive-thriftserver is because tests in the hive project use a shared singleton HiveContext and I cannot create a new one in the same project.

* 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.hive

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.{AnalysisException, SparkSession}
import org.apache.spark.util.Utils

class HiveMetastoreLazyInitializationSuite extends SparkFunSuite {

test("lazily initialize Hive client") {
val spark = SparkSession.builder()
.appName("HiveMetastoreLazyInitializationSuite")
.master("local[2]")
.enableHiveSupport()
.config("spark.hadoop.hive.metastore.uris", "thrift://127.0.0.1:11111")
.getOrCreate()
val originalLevel = org.apache.log4j.Logger.getRootLogger().getLevel
try {
// Avoid outputting a lot of expected warning logs
spark.sparkContext.setLogLevel("error")

// We should be able to run Spark jobs without Hive client.
assert(spark.sparkContext.range(0, 1).count() === 1)

// Make sure that we are not using the local derby metastore.
val exceptionString = Utils.exceptionString(intercept[AnalysisException] {
spark.sql("show tables")
})
for (msg <- Seq(
"show tables",
"Could not connect to meta store",
"org.apache.thrift.transport.TTransportException",
"Connection refused")) {
exceptionString.contains(msg)
}
} finally {
spark.sparkContext.setLogLevel(originalLevel.toString)
spark.stop()
}
}
}