Skip to content

Commit 0ee3379

Browse files
committed
Fixes ADD JAR
1 parent 1b5a9c8 commit 0ee3379

File tree

2 files changed

+12
-8
lines changed

2 files changed

+12
-8
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.client
1919

2020
import java.io.{File, PrintStream}
2121
import java.util.{Map => JMap}
22-
import javax.annotation.concurrent.GuardedBy
2322

2423
import scala.collection.JavaConverters._
2524
import scala.language.reflectiveCalls
@@ -548,7 +547,15 @@ private[hive] class ClientWrapper(
548547
}
549548

550549
def addJar(path: String): Unit = {
551-
clientLoader.addJar(path)
550+
val uri = new Path(path).toUri
551+
val jarURL = if (uri.getScheme == null) {
552+
// `path` is a local file path without a URL scheme
553+
new File(path).toURI.toURL
554+
} else {
555+
// `path` is a URL with a scheme
556+
uri.toURL
557+
}
558+
clientLoader.addJar(jarURL)
552559
runSqlHive(s"ADD JAR $path")
553560
}
554561

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -22,18 +22,16 @@ import java.lang.reflect.InvocationTargetException
2222
import java.net.{URL, URLClassLoader}
2323
import java.util
2424

25-
import scala.collection.mutable
2625
import scala.language.reflectiveCalls
2726
import scala.util.Try
2827

2928
import org.apache.commons.io.{FileUtils, IOUtils}
3029

3130
import org.apache.spark.Logging
3231
import org.apache.spark.deploy.SparkSubmitUtils
33-
import org.apache.spark.util.{MutableURLClassLoader, Utils}
34-
3532
import org.apache.spark.sql.catalyst.util.quietly
3633
import org.apache.spark.sql.hive.HiveContext
34+
import org.apache.spark.util.{MutableURLClassLoader, Utils}
3735

3836
/** Factory for `IsolatedClientLoader` with specific versions of hive. */
3937
private[hive] object IsolatedClientLoader {
@@ -190,9 +188,8 @@ private[hive] class IsolatedClientLoader(
190188
new NonClosableMutableURLClassLoader(isolatedClassLoader)
191189
}
192190

193-
private[hive] def addJar(path: String): Unit = synchronized {
194-
val jarURL = new java.io.File(path).toURI.toURL
195-
classLoader.addURL(jarURL)
191+
private[hive] def addJar(path: URL): Unit = synchronized {
192+
classLoader.addURL(path)
196193
}
197194

198195
/** The isolated client interface to Hive. */

0 commit comments

Comments
 (0)