diff options
author | Michael Armbrust <michael@databricks.com> | 2014-11-16 15:05:04 -0800 |
---|---|---|
committer | Michael Armbrust <michael@databricks.com> | 2014-11-16 15:05:08 -0800 |
commit | 45ce3273cb618d14ec4d20c4c95699634b951086 (patch) | |
tree | 100aa5c6368cd63bb896e36f3bfca5ed5e29279b /sql/hive-thriftserver | |
parent | cb6bd83a91d9b4a227dc6467255231869c1820e2 (diff) | |
download | spark-45ce3273cb618d14ec4d20c4c95699634b951086.tar.gz spark-45ce3273cb618d14ec4d20c4c95699634b951086.tar.bz2 spark-45ce3273cb618d14ec4d20c4c95699634b951086.zip |
Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
Author: Michael Armbrust <michael@databricks.com>
Closes #3292 from marmbrus/revert4309 and squashes the following commits:
808e96e [Michael Armbrust] Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
Diffstat (limited to 'sql/hive-thriftserver')
3 files changed, 47 insertions, 83 deletions
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 23d12cbff3..bba29b2bdc 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File import java.net.ServerSocket -import java.sql.{Date, DriverManager, Statement} +import java.sql.{DriverManager, Statement} import java.util.concurrent.TimeoutException -import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.concurrent.{Await, Promise} @@ -52,15 +51,6 @@ import org.apache.spark.sql.hive.HiveShim class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) - object TestData { - def getTestDataFilePath(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") - } - - val smallKv = getTestDataFilePath("small_kv.txt") - val smallKvWithNull = getTestDataFilePath("small_kv_with_null.txt") - } - def randomListeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. @@ -155,8 +145,12 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } } - // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths - val env = Seq("SPARK_TESTING" -> "0") + val env = Seq( + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + "SPARK_TESTING" -> "0", + // Prevents loading classes out of the assembly jar. Otherwise Utils.sparkVersion can't read + // proper version information from the jar manifest. + "SPARK_PREPEND_CLASSES" -> "") Process(command, None, env: _*).run(ProcessLogger( captureThriftServerOutput("stdout"), @@ -200,12 +194,15 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("Test JDBC query execution") { withJdbcStatement() { statement => - val queries = Seq( - "SET spark.sql.shuffle.partitions=3", - "DROP TABLE IF EXISTS test", - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test", - "CACHE TABLE test") + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + val queries = + s"""SET spark.sql.shuffle.partitions=3; + |CREATE TABLE test(key INT, val STRING); + |LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test; + |CACHE TABLE test; + """.stripMargin.split(";").map(_.trim).filter(_.nonEmpty) queries.foreach(statement.execute) @@ -219,10 +216,14 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("SPARK-3004 regression: result set containing NULL") { withJdbcStatement() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource( + "data/files/small_kv_with_null.txt") + val queries = Seq( "DROP TABLE IF EXISTS test_null", "CREATE TABLE test_null(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKvWithNull}' OVERWRITE INTO TABLE test_null") + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") queries.foreach(statement.execute) @@ -269,10 +270,13 @@ class HiveThriftServer2Suite extends FunSuite with Logging { test("SPARK-4292 regression: result set iterator issue") { withJdbcStatement() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + val queries = Seq( "DROP TABLE IF EXISTS test_4292", "CREATE TABLE test_4292(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_4292") + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_4292") queries.foreach(statement.execute) @@ -280,52 +284,10 @@ class HiveThriftServer2Suite extends FunSuite with Logging { Seq(238, 86, 311, 27, 165).foreach { key => resultSet.next() - assert(resultSet.getInt(1) === key) + assert(resultSet.getInt(1) == key) } statement.executeQuery("DROP TABLE IF EXISTS test_4292") } } - - test("SPARK-4309 regression: Date type support") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_date", - "CREATE TABLE test_date(key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_date") - - queries.foreach(statement.execute) - - assertResult(Date.valueOf("2011-01-01")) { - val resultSet = statement.executeQuery( - "SELECT CAST('2011-01-01' as date) FROM test_date LIMIT 1") - resultSet.next() - resultSet.getDate(1) - } - } - } - - test("SPARK-4407 regression: Complex type support") { - withJdbcStatement() { statement => - val queries = Seq( - "DROP TABLE IF EXISTS test_map", - "CREATE TABLE test_map(key INT, value STRING)", - s"LOAD DATA LOCAL INPATH '${TestData.smallKv}' OVERWRITE INTO TABLE test_map") - - queries.foreach(statement.execute) - - assertResult("""{238:"val_238"}""") { - val resultSet = statement.executeQuery("SELECT MAP(key, value) FROM test_map LIMIT 1") - resultSet.next() - resultSet.getString(1) - } - - assertResult("""["238","val_238"]""") { - val resultSet = statement.executeQuery( - "SELECT ARRAY(CAST(key AS STRING), value) FROM test_map LIMIT 1") - resultSet.next() - resultSet.getString(1) - } - } - } } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 9258ad0cdf..aa2e3cab72 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import java.util.{ArrayList => JArrayList, Map => JMap} import scala.collection.JavaConversions._ @@ -131,13 +131,14 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) case ShortType => to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) - case DateType => - to.addColumnValue(ColumnValue.dateValue(from(ordinal).asInstanceOf[Date])) case TimestampType => to.addColumnValue( ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((from.get(ordinal), dataTypes(ordinal))) to.addColumnValue(ColumnValue.stringValue(hiveString)) } } @@ -162,8 +163,6 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.byteValue(null)) case ShortType => to.addColumnValue(ColumnValue.shortValue(null)) - case DateType => - to.addColumnValue(ColumnValue.dateValue(null)) case TimestampType => to.addColumnValue(ColumnValue.timestampValue(null)) case BinaryType | _: ArrayType | _: StructType | _: MapType => diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 3c7f62af45..a642478d08 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.security.PrivilegedExceptionAction -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import java.util.concurrent.Future import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} @@ -113,7 +113,7 @@ private[hive] class SparkExecuteStatementOperation( def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { dataTypes(ordinal) match { case StringType => - to += from.getString(ordinal) + to += from.get(ordinal).asInstanceOf[String] case IntegerType => to += from.getInt(ordinal) case BooleanType => @@ -123,20 +123,23 @@ private[hive] class SparkExecuteStatementOperation( case FloatType => to += from.getFloat(ordinal) case DecimalType() => - to += from.getAs[BigDecimal](ordinal).bigDecimal + to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal case LongType => to += from.getLong(ordinal) case ByteType => to += from.getByte(ordinal) case ShortType => to += from.getShort(ordinal) - case DateType => - to += from.getAs[Date](ordinal) case TimestampType => - to += from.getAs[Timestamp](ordinal) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = HiveContext.toHiveString((from.get(ordinal), dataTypes(ordinal))) - to += hiveString + to += from.get(ordinal).asInstanceOf[Timestamp] + case BinaryType => + to += from.get(ordinal).asInstanceOf[String] + case _: ArrayType => + to += from.get(ordinal).asInstanceOf[String] + case _: StructType => + to += from.get(ordinal).asInstanceOf[String] + case _: MapType => + to += from.get(ordinal).asInstanceOf[String] } } @@ -144,9 +147,9 @@ private[hive] class SparkExecuteStatementOperation( validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) - val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) if (!iter.hasNext) { - resultRowSet + reultRowSet } else { // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int val maxRows = maxRowsL.toInt @@ -163,10 +166,10 @@ private[hive] class SparkExecuteStatementOperation( } curCol += 1 } - resultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) + reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) curRow += 1 } - resultRowSet + reultRowSet } } |