Skip to content

Commit

Permalink
Initialize UrlStreamHandlerFactory per jvm and fix typo.
Browse files Browse the repository at this point in the history
  • Loading branch information
jiangzho committed Apr 30, 2019
1 parent 5ddef27 commit 7a789a6
Showing 1 changed file with 18 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -42,18 +42,7 @@ import org.apache.spark.util.{MutableURLClassLoader, Utils}
*/
private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {

private var initialized = false

initialize()

private def initialize() {
synchronized {
if (!initialized) {
SharedState.setUrlStreamHanderFactoryIfNeeded(sparkContext.conf)
initialized = true
}
}
}
SharedState.setUrlStreamHandlerFactoryIfNeeded(sparkContext.conf)

// Load hive-site.xml into hadoopConf and determine the warehouse path we want to use, based on
// the config from both hive and Spark SQL. Finally set the warehouse config value to sparkConf.
Expand Down Expand Up @@ -166,17 +155,26 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
}

object SharedState extends Logging {
private var initialized = false

private def setUrlStreamHanderFactoryIfNeeded(conf: SparkConf) = {
try {
if (conf.getBoolean("spark.FsUrlStreamHandlerFactory.enabled", true)) {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
private def setUrlStreamHandlerFactoryIfNeeded(conf: SparkConf) = {
synchronized {
if (!initialized) {
try {
if (conf.getBoolean("spark.FsUrlStreamHandlerFactory.enabled", true)) {
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())
}
} catch {
case e: Error =>
logWarning("URL.setURLStreamHandlerFactory failed to set " +
"FsUrlStreamHandlerFactory", e)
} finally {
// don't retry on failure
initialized = true
}
}
}
} catch {
case e: Error =>
logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory")
}
}

private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog"

Expand Down

0 comments on commit 7a789a6

Please sign in to comment.