Skip to content
Snippets Groups Projects
Commit 12411b5e authored by Shixiong Zhu's avatar Shixiong Zhu Committed by gatorsmile
Browse files

[SPARK-21732][SQL] Lazily init hive metastore client

## What changes were proposed in this pull request?

This PR changes the codes to lazily init hive metastore client so that we can create SparkSession without talking to the hive metastore sever.

It's pretty helpful when you set a hive metastore server but it's down. You can still start the Spark shell to debug.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18944 from zsxwing/hive-lazy-init.
parent 0422ce06
No related branches found
No related tags found
No related merge requests found
......@@ -287,14 +287,14 @@ abstract class BaseSessionStateBuilder(
experimentalMethods,
functionRegistry,
udfRegistration,
catalog,
() => catalog,
sqlParser,
analyzer,
optimizer,
() => analyzer,
() => optimizer,
planner,
streamingQueryManager,
listenerManager,
resourceLoader,
() => resourceLoader,
createQueryExecution,
createClone)
}
......
......@@ -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.
*/
......@@ -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)
......
/*
* 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()
}
}
}
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment