Skip to content
Closed
Show file tree
Hide file tree
Changes from 3 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 @@ -41,6 +41,7 @@ import org.apache.thrift.transport.TSocket
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.sql.hive.test.TestHive
import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer
import org.apache.spark.util.Utils
import org.apache.spark.{Logging, SparkFunSuite}
Expand Down Expand Up @@ -462,6 +463,14 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
assert(conf.get("spark.sql.hive.version") === Some("1.2.1"))
}
}

test("SPARK-11191 add jar using path with URL scheme") {
withJdbcStatement { statement =>
val funcJar = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath
val jarURL = s"file:///$funcJar"
statement.executeQuery(s"ADD JAR $jarURL")
}
}
}

class HiveThriftHttpServerSuite extends HiveThriftJdbcTest {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@ package org.apache.spark.sql.hive.client

import java.io.{File, PrintStream}
import java.util.{Map => JMap}
import javax.annotation.concurrent.GuardedBy

import scala.collection.JavaConverters._
import scala.language.reflectiveCalls
Expand Down Expand Up @@ -548,7 +547,15 @@ private[hive] class ClientWrapper(
}

def addJar(path: String): Unit = {
clientLoader.addJar(path)
val uri = new Path(path).toUri

Choose a reason for hiding this comment

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

我在1.6.1 版本上测试了一下这次的修改,发现add jar 的方式的确是可以通过的。
但是 如果按照 CREATE FUNCTION xxx AS 'org.hue.udf.MyUpper' USING JAR 'hdfs://xxx/myudfs.jar';
的方式, 看log是现实了加载了 , jar。 但是实际运行时会报, class not found 。
请问这个bug 有思路么?
@liancheng

Choose a reason for hiding this comment

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

https://issues.apache.org/jira/browse/SPARK-16833, this isseue shows that sometimes it work.
looks like sometimes private val jars = new ConcurrentHashMap[String, File]() in class NettyStreamManager doesn't successful add jar.

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")
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,18 +22,16 @@ import java.lang.reflect.InvocationTargetException
import java.net.{URL, URLClassLoader}
import java.util

import scala.collection.mutable
import scala.language.reflectiveCalls
import scala.util.Try

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

import org.apache.spark.Logging
import org.apache.spark.deploy.SparkSubmitUtils
import org.apache.spark.util.{MutableURLClassLoader, Utils}

import org.apache.spark.sql.catalyst.util.quietly
import org.apache.spark.sql.hive.HiveContext
import org.apache.spark.util.{MutableURLClassLoader, Utils}

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

private[hive] def addJar(path: String): Unit = synchronized {
val jarURL = new java.io.File(path).toURI.toURL
classLoader.addURL(jarURL)
private[hive] def addJar(path: URL): Unit = synchronized {
classLoader.addURL(path)
}

/** The isolated client interface to Hive. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -927,7 +927,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {
test("SPARK-2263: Insert Map<K, V> values") {
sql("CREATE TABLE m(value MAP<INT, STRING>)")
sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")
sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map {
sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).foreach {
case (Row(map: Map[_, _]), Row(key: Int, value: String)) =>
assert(map.size === 1)
assert(map.head === (key, value))
Expand Down Expand Up @@ -961,10 +961,12 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter {

test("CREATE TEMPORARY FUNCTION") {
val funcJar = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath
sql(s"ADD JAR $funcJar")
val jarURL = s"file://$funcJar"
sql(s"ADD JAR $jarURL")
sql(
"""CREATE TEMPORARY FUNCTION udtf_count2 AS
| 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'""".stripMargin)
|'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
""".stripMargin)
assert(sql("DESCRIBE FUNCTION udtf_count2").count > 1)
sql("DROP TEMPORARY FUNCTION udtf_count2")
}
Expand Down