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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.connector.catalog.CatalogManager
import org.apache.spark.sql.execution._
import org.apache.spark.sql.streaming.StreamingQueryManager
import org.apache.spark.sql.util.ExecutionListenerManager
import org.apache.spark.util.DependencyUtils
import org.apache.spark.util.{DependencyUtils, Utils}

/**
* A class that holds all session-specific state in a given [[SparkSession]].
Expand Down Expand Up @@ -171,7 +171,7 @@ class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoade
* [[SessionState]].
*/
def addJar(path: String): Unit = {
val uri = URI.create(path)
val uri = Utils.resolveURI(path)
resolveJars(uri).foreach { p =>
session.sparkContext.addJar(p)
val uri = new Path(p).toUri
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark.sql.hive

import java.net.URI

import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog}
import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener
Expand All @@ -34,6 +32,7 @@ import org.apache.spark.sql.execution.streaming.ResolveWriteToStream
import org.apache.spark.sql.hive.client.HiveClient
import org.apache.spark.sql.hive.execution.PruneHiveTablePartitions
import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState}
import org.apache.spark.util.Utils

/**
* Builder that produces a Hive-aware `SessionState`.
Expand Down Expand Up @@ -126,7 +125,7 @@ class HiveSessionResourceLoader(
extends SessionResourceLoader(session) {
private lazy val client = clientBuilder()
override def addJar(path: String): Unit = {
val uri = URI.create(path)
val uri = Utils.resolveURI(path)
resolveJars(uri).foreach { p =>
client.addJar(p)
super.addJar(p)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.sql.hive.client

import java.io.{File, PrintStream}
import java.io.PrintStream
import java.lang.{Iterable => JIterable}
import java.lang.reflect.InvocationTargetException
import java.nio.charset.StandardCharsets.UTF_8
Expand Down Expand Up @@ -954,16 +954,8 @@ private[hive] class HiveClientImpl(
}

def addJar(path: String): Unit = {
val uri = new Path(path).toUri
val jarURL = if (uri.getScheme == null) {
// `path` is a local file path without a URL scheme
new File(path).toURI.toURL
} else {
// `path` is a URL with a scheme
uri.toURL
}
clientLoader.addJar(jarURL)
runSqlHive(s"ADD JAR $path")
val jarURI = Utils.resolveURI(path)
clientLoader.addJar(jarURI.toURL)
}

def newSession(): HiveClientImpl = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -830,6 +830,18 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
assert(sql(s"list jar $testJar").count() == 1)
}

test("SPARK-34955: ADD JAR should treat paths which contains white spaces") {
withTempDir { dir =>
val file = File.createTempFile("someprefix1", "somesuffix1", dir)
Files.write(file.toPath, "test_file1".getBytes)
val jarFile = new File(dir, "test file.jar")
TestUtils.createJar(Seq(file), jarFile)
sql(s"ADD JAR ${jarFile.getAbsolutePath}")
assert(sql("LIST JARS").
filter(_.getString(0).contains(s"${jarFile.getName}".replace(" ", "%20"))).count() > 0)
}
}

test("CREATE TEMPORARY FUNCTION") {
val funcJar = TestHive.getHiveFile("TestUDTF.jar")
val jarURL = funcJar.toURI.toURL
Expand Down