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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils,
import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.exec.Utilities
import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory}
import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory}
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.shims.ShimLoader
import org.apache.thrift.transport.TSocket
Expand Down Expand Up @@ -264,7 +264,9 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging {
val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf)

if (proc != null) {
if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) {
if (proc.isInstanceOf[Driver]
|| proc.isInstanceOf[SetProcessor]
|| proc.isInstanceOf[AddResourceProcessor]) {
val driver = new SparkSQLDriver

driver.init()
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -123,4 +123,28 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging {
test("Single command with -e") {
runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK")
}

test("SPARK-6392:add jar") {
val dataFilePath =
Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt")
val jarPath =
Thread.currentThread().getContextClassLoader.getResource("jar/TestUdf.jar").toString.split(":")(1)

runCliWithin(3.minute)(
s"""
|CREATE TABLE hive_test(key INT, val STRING);
|LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;
|""".stripMargin
-> "OK",
s"""
|add jar $jarPath ;
|create temporary function hello as 'hello';
|""".stripMargin
-> "Time taken: ",
"select hello(count(*)) from hive_test;"
-> "hello 5",
"DROP TABLE hive_test;"
-> "Time taken: "
)
}
}