aboutsummaryrefslogtreecommitdiff
path: root/sql/hive-thriftserver/src/test/scala/org/apache/spark
diff options
context:
space:
mode:
Diffstat (limited to 'sql/hive-thriftserver/src/test/scala/org/apache/spark')
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala20
-rw-r--r--sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala98
2 files changed, 65 insertions, 53 deletions
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index 8e1ebe2937..eb49eabcb1 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -162,7 +162,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
runCliWithin(3.minute)(
"CREATE TABLE hive_test(key INT, val STRING);"
- -> "OK",
+ -> "",
"SHOW TABLES;"
-> "hive_test",
s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;"
@@ -172,22 +172,22 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
"SELECT COUNT(*) FROM hive_test;"
-> "5",
"DROP TABLE hive_test;"
- -> "OK"
+ -> ""
)
}
test("Single command with -e") {
- runCliWithin(2.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK")
+ runCliWithin(2.minute, Seq("-e", "SHOW DATABASES;"))("" -> "")
}
test("Single command with --database") {
runCliWithin(2.minute)(
"CREATE DATABASE hive_test_db;"
- -> "OK",
+ -> "",
"USE hive_test_db;"
-> "",
"CREATE TABLE hive_test(key INT, val STRING);"
- -> "OK",
+ -> "",
"SHOW TABLES;"
-> "hive_test"
)
@@ -210,9 +210,9 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
"""CREATE TABLE t1(key string, val string)
|ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe';
""".stripMargin
- -> "OK",
+ -> "",
"CREATE TABLE sourceTable (key INT, val STRING);"
- -> "OK",
+ -> "",
s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE sourceTable;"
-> "OK",
"INSERT INTO TABLE t1 SELECT key, val FROM sourceTable;"
@@ -220,9 +220,9 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
"SELECT count(key) FROM t1;"
-> "5",
"DROP TABLE t1;"
- -> "OK",
+ -> "",
"DROP TABLE sourceTable;"
- -> "OK"
+ -> ""
)
}
@@ -230,7 +230,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
runCliWithin(timeout = 2.minute,
errorResponses = Seq("AnalysisException"))(
"select * from nonexistent_table;"
- -> "Error in query: Table not found: nonexistent_table;"
+ -> "Error in query: Table or View not found: nonexistent_table;"
)
}
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
index 33af624cfd..a1268b8e94 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala
@@ -491,46 +491,50 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest {
test("SPARK-11595 ADD JAR with input path having URL scheme") {
withJdbcStatement { statement =>
- val jarPath = "../hive/src/test/resources/TestUDTF.jar"
- val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath"
+ try {
+ val jarPath = "../hive/src/test/resources/TestUDTF.jar"
+ val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath"
- Seq(
- s"ADD JAR $jarURL",
- s"""CREATE TEMPORARY FUNCTION udtf_count2
- |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
- """.stripMargin
- ).foreach(statement.execute)
+ Seq(
+ s"ADD JAR $jarURL",
+ s"""CREATE TEMPORARY FUNCTION udtf_count2
+ |AS 'org.apache.spark.sql.hive.execution.GenericUDTFCount2'
+ """.stripMargin
+ ).foreach(statement.execute)
- val rs1 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2")
+ val rs1 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2")
- assert(rs1.next())
- assert(rs1.getString(1) === "Function: udtf_count2")
+ assert(rs1.next())
+ assert(rs1.getString(1) === "Function: udtf_count2")
- assert(rs1.next())
- assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") {
- rs1.getString(1)
- }
+ assert(rs1.next())
+ assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") {
+ rs1.getString(1)
+ }
- assert(rs1.next())
- assert(rs1.getString(1) === "Usage: To be added.")
+ assert(rs1.next())
+ assert(rs1.getString(1) === "Usage: To be added.")
- val dataPath = "../hive/src/test/resources/data/files/kv1.txt"
+ val dataPath = "../hive/src/test/resources/data/files/kv1.txt"
- Seq(
- s"CREATE TABLE test_udtf(key INT, value STRING)",
- s"LOAD DATA LOCAL INPATH '$dataPath' OVERWRITE INTO TABLE test_udtf"
- ).foreach(statement.execute)
+ Seq(
+ s"CREATE TABLE test_udtf(key INT, value STRING)",
+ s"LOAD DATA LOCAL INPATH '$dataPath' OVERWRITE INTO TABLE test_udtf"
+ ).foreach(statement.execute)
- val rs2 = statement.executeQuery(
- "SELECT key, cc FROM test_udtf LATERAL VIEW udtf_count2(value) dd AS cc")
+ val rs2 = statement.executeQuery(
+ "SELECT key, cc FROM test_udtf LATERAL VIEW udtf_count2(value) dd AS cc")
- assert(rs2.next())
- assert(rs2.getInt(1) === 97)
- assert(rs2.getInt(2) === 500)
+ assert(rs2.next())
+ assert(rs2.getInt(1) === 97)
+ assert(rs2.getInt(2) === 500)
- assert(rs2.next())
- assert(rs2.getInt(1) === 97)
- assert(rs2.getInt(2) === 500)
+ assert(rs2.next())
+ assert(rs2.getInt(1) === 97)
+ assert(rs2.getInt(2) === 500)
+ } finally {
+ statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2")
+ }
}
}
@@ -565,24 +569,28 @@ class SingleSessionSuite extends HiveThriftJdbcTest {
},
{ statement =>
- val rs1 = statement.executeQuery("SET foo")
+ try {
+ val rs1 = statement.executeQuery("SET foo")
- assert(rs1.next())
- assert(rs1.getString(1) === "foo")
- assert(rs1.getString(2) === "bar")
+ assert(rs1.next())
+ assert(rs1.getString(1) === "foo")
+ assert(rs1.getString(2) === "bar")
- val rs2 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2")
+ val rs2 = statement.executeQuery("DESCRIBE FUNCTION udtf_count2")
- assert(rs2.next())
- assert(rs2.getString(1) === "Function: udtf_count2")
+ assert(rs2.next())
+ assert(rs2.getString(1) === "Function: udtf_count2")
- assert(rs2.next())
- assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") {
- rs2.getString(1)
- }
+ assert(rs2.next())
+ assertResult("Class: org.apache.spark.sql.hive.execution.GenericUDTFCount2") {
+ rs2.getString(1)
+ }
- assert(rs2.next())
- assert(rs2.getString(1) === "Usage: To be added.")
+ assert(rs2.next())
+ assert(rs2.getString(1) === "Usage: To be added.")
+ } finally {
+ statement.executeQuery("DROP TEMPORARY FUNCTION udtf_count2")
+ }
}
)
}
@@ -763,11 +771,15 @@ abstract class HiveThriftServer2Test extends SparkFunSuite with BeforeAndAfterAl
extraEnvironment = Map(
// Disables SPARK_TESTING to exclude log4j.properties in test directories.
"SPARK_TESTING" -> "0",
+ // But set SPARK_SQL_TESTING to make spark-class happy.
+ "SPARK_SQL_TESTING" -> "1",
// Points SPARK_PID_DIR to SPARK_HOME, otherwise only 1 Thrift server instance can be
// started at a time, which is not Jenkins friendly.
"SPARK_PID_DIR" -> pidDir.getCanonicalPath),
redirectStderr = true)
+ logInfo(s"COMMAND: $command")
+ logInfo(s"OUTPUT: $lines")
lines.split("\n").collectFirst {
case line if line.contains(LOG_FILE_MARK) => new File(line.drop(LOG_FILE_MARK.length))
}.getOrElse {