[SPARK-7684] [SQL] Refactoring MetastoreDataSourcesSuite to workaround SPARK-7684
As stated in SPARK-7684, currently `TestHive.reset` has some execution order specific bug, which makes running specific test suites locally pretty frustrating. This PR refactors `MetastoreDataSourcesSuite` (which relies on `TestHive.reset` heavily) using various `withXxx` utility methods in `SQLTestUtils` to ask each test case to cleanup their own mess so that we can avoid calling `TestHive.reset`.
Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>
Closes #6353 from liancheng/workaround-spark-7684 and squashes the following commits:
26939aa [Yin Huai] Move the initialization of jsonFilePath to beforeAll.
a423d48 [Cheng Lian] Fixes Scala style issue
dfe45d0 [Cheng Lian] Refactors MetastoreDataSourcesSuite to workaround SPARK-7684
92a116d [Cheng Lian] Fixes minor styling issues
(cherry picked from commit b97ddff000
)
Signed-off-by: Yin Huai <yhuai@databricks.com>
This commit is contained in:
parent
d33142fd8c
commit
89fe93fc3b
|
@ -67,6 +67,10 @@ class QueryTest extends PlanTest {
|
||||||
checkAnswer(df, Seq(expectedAnswer))
|
checkAnswer(df, Seq(expectedAnswer))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected def checkAnswer(df: DataFrame, expectedAnswer: DataFrame): Unit = {
|
||||||
|
checkAnswer(df, expectedAnswer.collect())
|
||||||
|
}
|
||||||
|
|
||||||
def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) {
|
def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) {
|
||||||
test(sqlString) {
|
test(sqlString) {
|
||||||
checkAnswer(sqlContext.sql(sqlString), expectedAnswer)
|
checkAnswer(sqlContext.sql(sqlString), expectedAnswer)
|
||||||
|
|
|
@ -75,14 +75,18 @@ trait SQLTestUtils {
|
||||||
/**
|
/**
|
||||||
* Drops temporary table `tableName` after calling `f`.
|
* Drops temporary table `tableName` after calling `f`.
|
||||||
*/
|
*/
|
||||||
protected def withTempTable(tableName: String)(f: => Unit): Unit = {
|
protected def withTempTable(tableNames: String*)(f: => Unit): Unit = {
|
||||||
try f finally sqlContext.dropTempTable(tableName)
|
try f finally tableNames.foreach(sqlContext.dropTempTable)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Drops table `tableName` after calling `f`.
|
* Drops table `tableName` after calling `f`.
|
||||||
*/
|
*/
|
||||||
protected def withTable(tableName: String)(f: => Unit): Unit = {
|
protected def withTable(tableNames: String*)(f: => Unit): Unit = {
|
||||||
try f finally sqlContext.sql(s"DROP TABLE IF EXISTS $tableName")
|
try f finally {
|
||||||
|
tableNames.foreach { name =>
|
||||||
|
sqlContext.sql(s"DROP TABLE IF EXISTS $name")
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,124 +21,134 @@ import java.io.File
|
||||||
|
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
|
||||||
|
import org.scalatest.BeforeAndAfterAll
|
||||||
|
|
||||||
import org.apache.hadoop.fs.Path
|
import org.apache.hadoop.fs.Path
|
||||||
import org.apache.hadoop.mapred.InvalidInputException
|
import org.apache.hadoop.mapred.InvalidInputException
|
||||||
import org.scalatest.BeforeAndAfterEach
|
|
||||||
|
|
||||||
import org.apache.spark.sql._
|
import org.apache.spark.sql._
|
||||||
import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable}
|
import org.apache.spark.sql.hive.client.{HiveTable, ManagedTable}
|
||||||
|
import org.apache.spark.sql.hive.test.TestHive
|
||||||
import org.apache.spark.sql.hive.test.TestHive._
|
import org.apache.spark.sql.hive.test.TestHive._
|
||||||
import org.apache.spark.sql.hive.test.TestHive.implicits._
|
import org.apache.spark.sql.hive.test.TestHive.implicits._
|
||||||
import org.apache.spark.sql.parquet.ParquetRelation2
|
import org.apache.spark.sql.parquet.ParquetRelation2
|
||||||
import org.apache.spark.sql.sources.LogicalRelation
|
import org.apache.spark.sql.sources.LogicalRelation
|
||||||
|
import org.apache.spark.sql.test.SQLTestUtils
|
||||||
import org.apache.spark.sql.types._
|
import org.apache.spark.sql.types._
|
||||||
import org.apache.spark.util.Utils
|
import org.apache.spark.util.Utils
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests for persisting tables created though the data sources API into the metastore.
|
* Tests for persisting tables created though the data sources API into the metastore.
|
||||||
*/
|
*/
|
||||||
class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with BeforeAndAfterAll {
|
||||||
|
override val sqlContext = TestHive
|
||||||
|
|
||||||
override def afterEach(): Unit = {
|
var jsonFilePath: String = _
|
||||||
reset()
|
|
||||||
Utils.deleteRecursively(tempPath)
|
override def beforeAll(): Unit = {
|
||||||
|
jsonFilePath = Utils.getSparkClassLoader.getResource("sample.json").getFile
|
||||||
}
|
}
|
||||||
|
|
||||||
val filePath = Utils.getSparkClassLoader.getResource("sample.json").getFile
|
|
||||||
var tempPath: File = Utils.createTempDir()
|
|
||||||
tempPath.delete()
|
|
||||||
|
|
||||||
test("persistent JSON table") {
|
test("persistent JSON table") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
read.json(filePath).collect().toSeq)
|
read.json(jsonFilePath).collect().toSeq)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("persistent JSON table with a user specified schema") {
|
test("persistent JSON table with a user specified schema") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable (
|
||||||
|CREATE TABLE jsonTable (
|
|
||||||
|a string,
|
|a string,
|
||||||
|b String,
|
|b String,
|
||||||
|`c_!@(3)` int,
|
|`c_!@(3)` int,
|
||||||
|`<d>` Struct<`d!`:array<int>, `=`:array<struct<Dd2: boolean>>>)
|
|`<d>` Struct<`d!`:array<int>, `=`:array<struct<Dd2: boolean>>>)
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
read.json(filePath).registerTempTable("expectedJsonTable")
|
withTempTable("expectedJsonTable") {
|
||||||
|
read.json(jsonFilePath).registerTempTable("expectedJsonTable")
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT a, b, `c_!@(3)`, `<d>`.`d!`, `<d>`.`=` FROM jsonTable"),
|
sql("SELECT a, b, `c_!@(3)`, `<d>`.`d!`, `<d>`.`=` FROM jsonTable"),
|
||||||
sql("SELECT a, b, `c_!@(3)`, `<d>`.`d!`, `<d>`.`=` FROM expectedJsonTable").collect().toSeq)
|
sql("SELECT a, b, `c_!@(3)`, `<d>`.`d!`, `<d>`.`=` FROM expectedJsonTable"))
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("persistent JSON table with a user specified schema with a subset of fields") {
|
test("persistent JSON table with a user specified schema with a subset of fields") {
|
||||||
|
withTable("jsonTable") {
|
||||||
// This works because JSON objects are self-describing and JSONRelation can get needed
|
// This works because JSON objects are self-describing and JSONRelation can get needed
|
||||||
// field values based on field names.
|
// field values based on field names.
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable (`<d>` Struct<`=`:array<struct<Dd2: boolean>>>, b String)
|
||||||
|CREATE TABLE jsonTable (`<d>` Struct<`=`:array<struct<Dd2: boolean>>>, b String)
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
val innerStruct = StructType(
|
val innerStruct = StructType(Seq(
|
||||||
StructField("=", ArrayType(StructType(StructField("Dd2", BooleanType, true) :: Nil))) :: Nil)
|
StructField("=", ArrayType(StructType(StructField("Dd2", BooleanType, true) :: Nil)))))
|
||||||
val expectedSchema = StructType(
|
|
||||||
StructField("<d>", innerStruct, true) ::
|
val expectedSchema = StructType(Seq(
|
||||||
StructField("b", StringType, true) :: Nil)
|
StructField("<d>", innerStruct, true),
|
||||||
|
StructField("b", StringType, true)))
|
||||||
|
|
||||||
assert(expectedSchema === table("jsonTable").schema)
|
assert(expectedSchema === table("jsonTable").schema)
|
||||||
|
|
||||||
read.json(filePath).registerTempTable("expectedJsonTable")
|
withTempTable("expectedJsonTable") {
|
||||||
|
read.json(jsonFilePath).registerTempTable("expectedJsonTable")
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT b, `<d>`.`=` FROM jsonTable"),
|
sql("SELECT b, `<d>`.`=` FROM jsonTable"),
|
||||||
sql("SELECT b, `<d>`.`=` FROM expectedJsonTable").collect().toSeq)
|
sql("SELECT b, `<d>`.`=` FROM expectedJsonTable"))
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("resolve shortened provider names") {
|
test("resolve shortened provider names") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""
|
||||||
|CREATE TABLE jsonTable
|
|CREATE TABLE jsonTable
|
||||||
|USING org.apache.spark.sql.json
|
|USING org.apache.spark.sql.json
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
read.json(filePath).collect().toSeq)
|
read.json(jsonFilePath).collect().toSeq)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("drop table") {
|
test("drop table") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""
|
||||||
|CREATE TABLE jsonTable
|
|CREATE TABLE jsonTable
|
||||||
|USING org.apache.spark.sql.json
|
|USING org.apache.spark.sql.json
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
read.json(filePath).collect().toSeq)
|
read.json(jsonFilePath))
|
||||||
|
|
||||||
sql("DROP TABLE jsonTable")
|
sql("DROP TABLE jsonTable")
|
||||||
|
|
||||||
|
@ -147,20 +157,19 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
}
|
}
|
||||||
|
|
||||||
assert(
|
assert(
|
||||||
(new File(filePath)).exists(),
|
new File(jsonFilePath).exists(),
|
||||||
"The table with specified path is considered as an external table, " +
|
"The table with specified path is considered as an external table, " +
|
||||||
"its data should not deleted after DROP TABLE.")
|
"its data should not deleted after DROP TABLE.")
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("check change without refresh") {
|
test("check change without refresh") {
|
||||||
val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir())
|
withTempPath { tempDir =>
|
||||||
tempDir.delete()
|
withTable("jsonTable") {
|
||||||
sparkContext.parallelize(("a", "b") :: Nil).toDF()
|
(("a", "b") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
||||||
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
|
||||||
|
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json
|
|USING org.apache.spark.sql.json
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempDir.getCanonicalPath}'
|
| path '${tempDir.getCanonicalPath}'
|
||||||
|
@ -172,8 +181,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
Row("a", "b"))
|
Row("a", "b"))
|
||||||
|
|
||||||
Utils.deleteRecursively(tempDir)
|
Utils.deleteRecursively(tempDir)
|
||||||
sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF()
|
(("a1", "b1", "c1") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
||||||
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
|
||||||
|
|
||||||
// Schema is cached so the new column does not show. The updated values in existing columns
|
// Schema is cached so the new column does not show. The updated values in existing columns
|
||||||
// will show.
|
// will show.
|
||||||
|
@ -187,18 +195,17 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
Row("a1", "b1", "c1"))
|
Row("a1", "b1", "c1"))
|
||||||
Utils.deleteRecursively(tempDir)
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("drop, change, recreate") {
|
test("drop, change, recreate") {
|
||||||
val tempDir = File.createTempFile("sparksql", "json", Utils.createTempDir())
|
withTempPath { tempDir =>
|
||||||
tempDir.delete()
|
(("a", "b") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
||||||
sparkContext.parallelize(("a", "b") :: Nil).toDF()
|
|
||||||
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
|
||||||
|
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json
|
|USING org.apache.spark.sql.json
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempDir.getCanonicalPath}'
|
| path '${tempDir.getCanonicalPath}'
|
||||||
|
@ -210,14 +217,12 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
Row("a", "b"))
|
Row("a", "b"))
|
||||||
|
|
||||||
Utils.deleteRecursively(tempDir)
|
Utils.deleteRecursively(tempDir)
|
||||||
sparkContext.parallelize(("a", "b", "c") :: Nil).toDF()
|
(("a", "b", "c") :: Nil).toDF().toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
||||||
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
|
|
||||||
|
|
||||||
sql("DROP TABLE jsonTable")
|
sql("DROP TABLE jsonTable")
|
||||||
|
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json
|
|USING org.apache.spark.sql.json
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempDir.getCanonicalPath}'
|
| path '${tempDir.getCanonicalPath}'
|
||||||
|
@ -228,20 +233,22 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
Row("a", "b", "c"))
|
Row("a", "b", "c"))
|
||||||
Utils.deleteRecursively(tempDir)
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("invalidate cache and reload") {
|
test("invalidate cache and reload") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable (`c_!@(3)` int)
|
||||||
|CREATE TABLE jsonTable (`c_!@(3)` int)
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
read.json(filePath).registerTempTable("expectedJsonTable")
|
withTempTable("expectedJsonTable") {
|
||||||
|
read.json(jsonFilePath).registerTempTable("expectedJsonTable")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM jsonTable"),
|
sql("SELECT * FROM jsonTable"),
|
||||||
|
@ -259,23 +266,25 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
assert(expectedSchema === table("jsonTable").schema)
|
assert(expectedSchema === table("jsonTable").schema)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("CTAS") {
|
test("CTAS") {
|
||||||
|
withTempPath { tempPath =>
|
||||||
|
withTable("jsonTable", "ctasJsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE ctasJsonTable
|
||||||
|CREATE TABLE ctasJsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempPath}'
|
| path '$tempPath'
|
||||||
|) AS
|
|) AS
|
||||||
|SELECT * FROM jsonTable
|
|SELECT * FROM jsonTable
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
@ -286,23 +295,27 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
sql("SELECT * FROM ctasJsonTable"),
|
sql("SELECT * FROM ctasJsonTable"),
|
||||||
sql("SELECT * FROM jsonTable").collect())
|
sql("SELECT * FROM jsonTable").collect())
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("CTAS with IF NOT EXISTS") {
|
test("CTAS with IF NOT EXISTS") {
|
||||||
|
withTempPath { path =>
|
||||||
|
val tempPath = path.getCanonicalPath
|
||||||
|
|
||||||
|
withTable("jsonTable", "ctasJsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE ctasJsonTable
|
||||||
|CREATE TABLE ctasJsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempPath}'
|
| path '$tempPath'
|
||||||
|) AS
|
|) AS
|
||||||
|SELECT * FROM jsonTable
|
|SELECT * FROM jsonTable
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
@ -310,27 +323,27 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
// Create the table again should trigger a AnalysisException.
|
// Create the table again should trigger a AnalysisException.
|
||||||
val message = intercept[AnalysisException] {
|
val message = intercept[AnalysisException] {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE ctasJsonTable
|
||||||
|CREATE TABLE ctasJsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempPath}'
|
| path '$tempPath'
|
||||||
|) AS
|
|) AS
|
||||||
|SELECT * FROM jsonTable
|
|SELECT * FROM jsonTable
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
}.getMessage
|
}.getMessage
|
||||||
assert(message.contains("Table ctasJsonTable already exists."),
|
|
||||||
|
assert(
|
||||||
|
message.contains("Table ctasJsonTable already exists."),
|
||||||
"We should complain that ctasJsonTable already exists")
|
"We should complain that ctasJsonTable already exists")
|
||||||
|
|
||||||
// The following statement should be fine if it has IF NOT EXISTS.
|
// The following statement should be fine if it has IF NOT EXISTS.
|
||||||
// It tries to create a table ctasJsonTable with a new schema.
|
// It tries to create a table ctasJsonTable with a new schema.
|
||||||
// The actual table's schema and data should not be changed.
|
// The actual table's schema and data should not be changed.
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE IF NOT EXISTS ctasJsonTable
|
||||||
|CREATE TABLE IF NOT EXISTS ctasJsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${tempPath}'
|
| path '$tempPath'
|
||||||
|) AS
|
|) AS
|
||||||
|SELECT a FROM jsonTable
|
|SELECT a FROM jsonTable
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
@ -345,14 +358,16 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
sql("SELECT * FROM ctasJsonTable"),
|
sql("SELECT * FROM ctasJsonTable"),
|
||||||
sql("SELECT * FROM jsonTable").collect())
|
sql("SELECT * FROM jsonTable").collect())
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("CTAS a managed table") {
|
test("CTAS a managed table") {
|
||||||
|
withTable("jsonTable", "ctasJsonTable", "loadedTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${filePath}'
|
| path '$jsonFilePath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
|
@ -363,8 +378,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
// It is a managed table when we do not specify the location.
|
// It is a managed table when we do not specify the location.
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE ctasJsonTable
|
||||||
|CREATE TABLE ctasJsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|AS
|
|AS
|
||||||
|SELECT * FROM jsonTable
|
|SELECT * FROM jsonTable
|
||||||
|
@ -373,11 +387,10 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
assert(fs.exists(filesystemPath), s"$expectedPath should exist after we create the table.")
|
assert(fs.exists(filesystemPath), s"$expectedPath should exist after we create the table.")
|
||||||
|
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE loadedTable
|
||||||
|CREATE TABLE loadedTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path '${expectedPath}'
|
| path '$expectedPath'
|
||||||
|)
|
|)
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
|
@ -385,17 +398,17 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM ctasJsonTable"),
|
sql("SELECT * FROM ctasJsonTable"),
|
||||||
sql("SELECT * FROM loadedTable").collect()
|
sql("SELECT * FROM loadedTable"))
|
||||||
)
|
|
||||||
|
|
||||||
sql("DROP TABLE ctasJsonTable")
|
sql("DROP TABLE ctasJsonTable")
|
||||||
assert(!fs.exists(filesystemPath), s"$expectedPath should not exist after we drop the table.")
|
assert(!fs.exists(filesystemPath), s"$expectedPath should not exist after we drop the table.")
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("SPARK-5286 Fail to drop an invalid table when using the data source API") {
|
test("SPARK-5286 Fail to drop an invalid table when using the data source API") {
|
||||||
|
withTable("jsonTable") {
|
||||||
sql(
|
sql(
|
||||||
s"""
|
s"""CREATE TABLE jsonTable
|
||||||
|CREATE TABLE jsonTable
|
|
||||||
|USING org.apache.spark.sql.json.DefaultSource
|
|USING org.apache.spark.sql.json.DefaultSource
|
||||||
|OPTIONS (
|
|OPTIONS (
|
||||||
| path 'it is not a path at all!'
|
| path 'it is not a path at all!'
|
||||||
|
@ -404,54 +417,50 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
sql("DROP TABLE jsonTable").collect().foreach(println)
|
sql("DROP TABLE jsonTable").collect().foreach(println)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("SPARK-5839 HiveMetastoreCatalog does not recognize table aliases of data source tables.") {
|
test("SPARK-5839 HiveMetastoreCatalog does not recognize table aliases of data source tables.") {
|
||||||
val originalDefaultSource = conf.defaultDataSourceName
|
withTable("savedJsonTable") {
|
||||||
|
// Save the df as a managed table (by not specifying the path).
|
||||||
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
|
(1 to 10)
|
||||||
val df = read.json(rdd)
|
.map(i => i -> s"str$i")
|
||||||
|
.toDF("a", "b")
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json")
|
.write
|
||||||
// Save the df as a managed table (by not specifiying the path).
|
.format("json")
|
||||||
df.write.saveAsTable("savedJsonTable")
|
.saveAsTable("savedJsonTable")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"),
|
sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"),
|
||||||
(1 to 4).map(i => Row(i, s"str${i}")))
|
(1 to 4).map(i => Row(i, s"str$i")))
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"),
|
sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"),
|
||||||
(6 to 10).map(i => Row(i, s"str${i}")))
|
(6 to 10).map(i => Row(i, s"str$i")))
|
||||||
|
|
||||||
invalidateTable("savedJsonTable")
|
invalidateTable("savedJsonTable")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"),
|
sql("SELECT * FROM savedJsonTable where savedJsonTable.a < 5"),
|
||||||
(1 to 4).map(i => Row(i, s"str${i}")))
|
(1 to 4).map(i => Row(i, s"str$i")))
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"),
|
sql("SELECT * FROM savedJsonTable tmp where tmp.a > 5"),
|
||||||
(6 to 10).map(i => Row(i, s"str${i}")))
|
(6 to 10).map(i => Row(i, s"str$i")))
|
||||||
|
}
|
||||||
// Drop table will also delete the data.
|
|
||||||
sql("DROP TABLE savedJsonTable")
|
|
||||||
|
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
test("save table") {
|
test("save table") {
|
||||||
val originalDefaultSource = conf.defaultDataSourceName
|
withTempPath { path =>
|
||||||
|
val tempPath = path.getCanonicalPath
|
||||||
|
|
||||||
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
|
withTable("savedJsonTable") {
|
||||||
val df = read.json(rdd)
|
val df = (1 to 10).map(i => i -> s"str$i").toDF("a", "b")
|
||||||
|
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json")
|
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
|
||||||
// Save the df as a managed table (by not specifiying the path).
|
// Save the df as a managed table (by not specifying the path).
|
||||||
df.write.saveAsTable("savedJsonTable")
|
df.write.saveAsTable("savedJsonTable")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(sql("SELECT * FROM savedJsonTable"), df)
|
||||||
sql("SELECT * FROM savedJsonTable"),
|
|
||||||
df.collect())
|
|
||||||
|
|
||||||
// Right now, we cannot append to an existing JSON table.
|
// Right now, we cannot append to an existing JSON table.
|
||||||
intercept[RuntimeException] {
|
intercept[RuntimeException] {
|
||||||
|
@ -460,159 +469,174 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
// We can overwrite it.
|
// We can overwrite it.
|
||||||
df.write.mode(SaveMode.Overwrite).saveAsTable("savedJsonTable")
|
df.write.mode(SaveMode.Overwrite).saveAsTable("savedJsonTable")
|
||||||
checkAnswer(
|
checkAnswer(sql("SELECT * FROM savedJsonTable"), df)
|
||||||
sql("SELECT * FROM savedJsonTable"),
|
|
||||||
df.collect())
|
|
||||||
|
|
||||||
// When the save mode is Ignore, we will do nothing when the table already exists.
|
// When the save mode is Ignore, we will do nothing when the table already exists.
|
||||||
df.select("b").write.mode(SaveMode.Ignore).saveAsTable("savedJsonTable")
|
df.select("b").write.mode(SaveMode.Ignore).saveAsTable("savedJsonTable")
|
||||||
assert(df.schema === table("savedJsonTable").schema)
|
assert(df.schema === table("savedJsonTable").schema)
|
||||||
checkAnswer(
|
checkAnswer(sql("SELECT * FROM savedJsonTable"), df)
|
||||||
sql("SELECT * FROM savedJsonTable"),
|
|
||||||
df.collect())
|
|
||||||
|
|
||||||
// Drop table will also delete the data.
|
// Drop table will also delete the data.
|
||||||
sql("DROP TABLE savedJsonTable")
|
sql("DROP TABLE savedJsonTable")
|
||||||
intercept[InvalidInputException] {
|
intercept[InvalidInputException] {
|
||||||
read.json(catalog.hiveDefaultTableFilePath("savedJsonTable"))
|
read.json(catalog.hiveDefaultTableFilePath("savedJsonTable"))
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
// Create an external table by specifying the path.
|
// Create an external table by specifying the path.
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name")
|
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
|
||||||
df.write
|
df.write
|
||||||
.format("org.apache.spark.sql.json")
|
.format("org.apache.spark.sql.json")
|
||||||
.mode(SaveMode.Append)
|
.mode(SaveMode.Append)
|
||||||
.option("path", tempPath.toString)
|
.option("path", tempPath.toString)
|
||||||
.saveAsTable("savedJsonTable")
|
.saveAsTable("savedJsonTable")
|
||||||
checkAnswer(
|
|
||||||
sql("SELECT * FROM savedJsonTable"),
|
checkAnswer(sql("SELECT * FROM savedJsonTable"), df)
|
||||||
df.collect())
|
}
|
||||||
|
|
||||||
// Data should not be deleted after we drop the table.
|
// Data should not be deleted after we drop the table.
|
||||||
sql("DROP TABLE savedJsonTable")
|
sql("DROP TABLE savedJsonTable")
|
||||||
checkAnswer(
|
checkAnswer(read.json(tempPath.toString), df)
|
||||||
read.json(tempPath.toString),
|
}
|
||||||
df.collect())
|
}
|
||||||
|
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
test("create external table") {
|
test("create external table") {
|
||||||
val originalDefaultSource = conf.defaultDataSourceName
|
withTempPath { tempPath =>
|
||||||
|
withTable("savedJsonTable", "createdJsonTable") {
|
||||||
|
val df = read.json(sparkContext.parallelize((1 to 10).map { i =>
|
||||||
|
s"""{ "a": $i, "b": "str$i" }"""
|
||||||
|
}))
|
||||||
|
|
||||||
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
|
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
|
||||||
val df = read.json(rdd)
|
df.write
|
||||||
|
.format("json")
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name")
|
|
||||||
df.write.format("org.apache.spark.sql.json")
|
|
||||||
.mode(SaveMode.Append)
|
.mode(SaveMode.Append)
|
||||||
.option("path", tempPath.toString)
|
.option("path", tempPath.toString)
|
||||||
.saveAsTable("savedJsonTable")
|
.saveAsTable("savedJsonTable")
|
||||||
|
}
|
||||||
|
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "org.apache.spark.sql.json")
|
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "json") {
|
||||||
createExternalTable("createdJsonTable", tempPath.toString)
|
createExternalTable("createdJsonTable", tempPath.toString)
|
||||||
assert(table("createdJsonTable").schema === df.schema)
|
assert(table("createdJsonTable").schema === df.schema)
|
||||||
checkAnswer(
|
checkAnswer(sql("SELECT * FROM createdJsonTable"), df)
|
||||||
sql("SELECT * FROM createdJsonTable"),
|
|
||||||
df.collect())
|
|
||||||
|
|
||||||
var message = intercept[AnalysisException] {
|
assert(
|
||||||
createExternalTable("createdJsonTable", filePath.toString)
|
intercept[AnalysisException] {
|
||||||
}.getMessage
|
createExternalTable("createdJsonTable", jsonFilePath.toString)
|
||||||
assert(message.contains("Table createdJsonTable already exists."),
|
}.getMessage.contains("Table createdJsonTable already exists."),
|
||||||
"We should complain that ctasJsonTable already exists")
|
"We should complain that createdJsonTable already exists")
|
||||||
|
}
|
||||||
|
|
||||||
// Data should not be deleted.
|
// Data should not be deleted.
|
||||||
sql("DROP TABLE createdJsonTable")
|
sql("DROP TABLE createdJsonTable")
|
||||||
checkAnswer(
|
checkAnswer(read.json(tempPath.toString), df)
|
||||||
read.json(tempPath.toString),
|
|
||||||
df.collect())
|
|
||||||
|
|
||||||
// Try to specify the schema.
|
// Try to specify the schema.
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, "not a source name")
|
withSQLConf(SQLConf.DEFAULT_DATA_SOURCE_NAME -> "not a source name") {
|
||||||
val schema = StructType(StructField("b", StringType, true) :: Nil)
|
val schema = StructType(StructField("b", StringType, true) :: Nil)
|
||||||
createExternalTable(
|
createExternalTable(
|
||||||
"createdJsonTable",
|
"createdJsonTable",
|
||||||
"org.apache.spark.sql.json",
|
"org.apache.spark.sql.json",
|
||||||
schema,
|
schema,
|
||||||
Map("path" -> tempPath.toString))
|
Map("path" -> tempPath.toString))
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT * FROM createdJsonTable"),
|
sql("SELECT * FROM createdJsonTable"),
|
||||||
sql("SELECT b FROM savedJsonTable").collect())
|
sql("SELECT b FROM savedJsonTable"))
|
||||||
|
|
||||||
sql("DROP TABLE createdJsonTable")
|
sql("DROP TABLE createdJsonTable")
|
||||||
|
|
||||||
message = intercept[RuntimeException] {
|
assert(
|
||||||
|
intercept[RuntimeException] {
|
||||||
createExternalTable(
|
createExternalTable(
|
||||||
"createdJsonTable",
|
"createdJsonTable",
|
||||||
"org.apache.spark.sql.json",
|
"org.apache.spark.sql.json",
|
||||||
schema,
|
schema,
|
||||||
Map.empty[String, String])
|
Map.empty[String, String])
|
||||||
}.getMessage
|
}.getMessage.contains("'path' must be specified for json data."),
|
||||||
assert(
|
|
||||||
message.contains("'path' must be specified for json data."),
|
|
||||||
"We should complain that path is not specified.")
|
"We should complain that path is not specified.")
|
||||||
|
}
|
||||||
sql("DROP TABLE savedJsonTable")
|
}
|
||||||
conf.setConf(SQLConf.DEFAULT_DATA_SOURCE_NAME, originalDefaultSource)
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (HiveShim.version == "0.13.1") {
|
if (HiveShim.version == "0.13.1") {
|
||||||
test("scan a parquet table created through a CTAS statement") {
|
test("scan a parquet table created through a CTAS statement") {
|
||||||
val originalConvertMetastore = getConf("spark.sql.hive.convertMetastoreParquet", "true")
|
withSQLConf(
|
||||||
val originalUseDataSource = getConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
|
"spark.sql.hive.convertMetastoreParquet" -> "true",
|
||||||
setConf("spark.sql.hive.convertMetastoreParquet", "true")
|
SQLConf.PARQUET_USE_DATA_SOURCE_API -> "true") {
|
||||||
setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, "true")
|
|
||||||
|
|
||||||
val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str${i}"}"""))
|
withTempTable("jt") {
|
||||||
read.json(rdd).registerTempTable("jt")
|
(1 to 10).map(i => i -> s"str$i").toDF("a", "b").registerTempTable("jt")
|
||||||
|
|
||||||
|
withTable("test_parquet_ctas") {
|
||||||
sql(
|
sql(
|
||||||
"""
|
"""CREATE TABLE test_parquet_ctas STORED AS PARQUET
|
||||||
|create table test_parquet_ctas STORED AS parquET
|
|AS SELECT tmp.a FROM jt tmp WHERE tmp.a < 5
|
||||||
|AS select tmp.a from jt tmp where tmp.a < 5
|
|
||||||
""".stripMargin)
|
""".stripMargin)
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql(s"SELECT a FROM test_parquet_ctas WHERE a > 2 "),
|
sql(s"SELECT a FROM test_parquet_ctas WHERE a > 2 "),
|
||||||
Row(3) :: Row(4) :: Nil
|
Row(3) :: Row(4) :: Nil)
|
||||||
)
|
|
||||||
|
|
||||||
table("test_parquet_ctas").queryExecution.optimizedPlan match {
|
table("test_parquet_ctas").queryExecution.optimizedPlan match {
|
||||||
case LogicalRelation(p: ParquetRelation2) => // OK
|
case LogicalRelation(p: ParquetRelation2) => // OK
|
||||||
case _ =>
|
case _ =>
|
||||||
fail(
|
fail(s"test_parquet_ctas should have be converted to ${classOf[ParquetRelation2]}")
|
||||||
"test_parquet_ctas should be converted to " +
|
}
|
||||||
s"${classOf[ParquetRelation2].getCanonicalName}")
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Clenup and reset confs.
|
|
||||||
sql("DROP TABLE IF EXISTS jt")
|
|
||||||
sql("DROP TABLE IF EXISTS test_parquet_ctas")
|
|
||||||
setConf("spark.sql.hive.convertMetastoreParquet", originalConvertMetastore)
|
|
||||||
setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalUseDataSource)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("Pre insert nullability check (ArrayType)") {
|
test("Pre insert nullability check (ArrayType)") {
|
||||||
val df1 =
|
withTable("arrayInParquet") {
|
||||||
createDataFrame(Tuple1(Seq(Int.box(1), null.asInstanceOf[Integer])) :: Nil).toDF("a")
|
{
|
||||||
val expectedSchema1 =
|
val df = (Tuple1(Seq(Int.box(1), null: Integer)) :: Nil).toDF("a")
|
||||||
|
val expectedSchema =
|
||||||
StructType(
|
StructType(
|
||||||
StructField("a", ArrayType(IntegerType, containsNull = true), nullable = true) :: Nil)
|
StructField(
|
||||||
assert(df1.schema === expectedSchema1)
|
"a",
|
||||||
df1.write.mode(SaveMode.Overwrite).format("parquet").saveAsTable("arrayInParquet")
|
ArrayType(IntegerType, containsNull = true),
|
||||||
|
nullable = true) :: Nil)
|
||||||
|
|
||||||
val df2 =
|
assert(df.schema === expectedSchema)
|
||||||
createDataFrame(Tuple1(Seq(2, 3)) :: Nil).toDF("a")
|
|
||||||
val expectedSchema2 =
|
df.write
|
||||||
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.saveAsTable("arrayInParquet")
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
val df = (Tuple1(Seq(2, 3)) :: Nil).toDF("a")
|
||||||
|
val expectedSchema =
|
||||||
StructType(
|
StructType(
|
||||||
StructField("a", ArrayType(IntegerType, containsNull = false), nullable = true) :: Nil)
|
StructField(
|
||||||
assert(df2.schema === expectedSchema2)
|
"a",
|
||||||
df2.write.mode(SaveMode.Append).insertInto("arrayInParquet")
|
ArrayType(IntegerType, containsNull = false),
|
||||||
createDataFrame(Tuple1(Seq(4, 5)) :: Nil).toDF("a").write.mode(SaveMode.Append)
|
nullable = true) :: Nil)
|
||||||
|
|
||||||
|
assert(df.schema === expectedSchema)
|
||||||
|
|
||||||
|
df.write
|
||||||
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.insertInto("arrayInParquet")
|
||||||
|
}
|
||||||
|
|
||||||
|
(Tuple1(Seq(4, 5)) :: Nil).toDF("a")
|
||||||
|
.write
|
||||||
|
.mode(SaveMode.Append)
|
||||||
.saveAsTable("arrayInParquet") // This one internally calls df2.insertInto.
|
.saveAsTable("arrayInParquet") // This one internally calls df2.insertInto.
|
||||||
createDataFrame(Tuple1(Seq(Int.box(6), null.asInstanceOf[Integer])) :: Nil).toDF("a").write
|
|
||||||
.mode(SaveMode.Append).saveAsTable("arrayInParquet")
|
(Tuple1(Seq(Int.box(6), null: Integer)) :: Nil).toDF("a")
|
||||||
|
.write
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.saveAsTable("arrayInParquet")
|
||||||
|
|
||||||
refreshTable("arrayInParquet")
|
refreshTable("arrayInParquet")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
|
@ -621,32 +645,57 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
Row(ArrayBuffer(2, 3)) ::
|
Row(ArrayBuffer(2, 3)) ::
|
||||||
Row(ArrayBuffer(4, 5)) ::
|
Row(ArrayBuffer(4, 5)) ::
|
||||||
Row(ArrayBuffer(6, null)) :: Nil)
|
Row(ArrayBuffer(6, null)) :: Nil)
|
||||||
|
}
|
||||||
sql("DROP TABLE arrayInParquet")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
test("Pre insert nullability check (MapType)") {
|
test("Pre insert nullability check (MapType)") {
|
||||||
val df1 =
|
withTable("mapInParquet") {
|
||||||
createDataFrame(Tuple1(Map(1 -> null.asInstanceOf[Integer])) :: Nil).toDF("a")
|
{
|
||||||
val mapType1 = MapType(IntegerType, IntegerType, valueContainsNull = true)
|
val df = (Tuple1(Map(1 -> (null: Integer))) :: Nil).toDF("a")
|
||||||
val expectedSchema1 =
|
val expectedSchema =
|
||||||
StructType(
|
StructType(
|
||||||
StructField("a", mapType1, nullable = true) :: Nil)
|
StructField(
|
||||||
assert(df1.schema === expectedSchema1)
|
"a",
|
||||||
df1.write.mode(SaveMode.Overwrite).format("parquet").saveAsTable("mapInParquet")
|
MapType(IntegerType, IntegerType, valueContainsNull = true),
|
||||||
|
nullable = true) :: Nil)
|
||||||
|
|
||||||
val df2 =
|
assert(df.schema === expectedSchema)
|
||||||
createDataFrame(Tuple1(Map(2 -> 3)) :: Nil).toDF("a")
|
|
||||||
val mapType2 = MapType(IntegerType, IntegerType, valueContainsNull = false)
|
df.write
|
||||||
val expectedSchema2 =
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Overwrite)
|
||||||
|
.saveAsTable("mapInParquet")
|
||||||
|
}
|
||||||
|
|
||||||
|
{
|
||||||
|
val df = (Tuple1(Map(2 -> 3)) :: Nil).toDF("a")
|
||||||
|
val expectedSchema =
|
||||||
StructType(
|
StructType(
|
||||||
StructField("a", mapType2, nullable = true) :: Nil)
|
StructField(
|
||||||
assert(df2.schema === expectedSchema2)
|
"a",
|
||||||
df2.write.mode(SaveMode.Append).insertInto("mapInParquet")
|
MapType(IntegerType, IntegerType, valueContainsNull = false),
|
||||||
createDataFrame(Tuple1(Map(4 -> 5)) :: Nil).toDF("a").write.mode(SaveMode.Append)
|
nullable = true) :: Nil)
|
||||||
|
|
||||||
|
assert(df.schema === expectedSchema)
|
||||||
|
|
||||||
|
df.write
|
||||||
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.insertInto("mapInParquet")
|
||||||
|
}
|
||||||
|
|
||||||
|
(Tuple1(Map(4 -> 5)) :: Nil).toDF("a")
|
||||||
|
.write
|
||||||
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Append)
|
||||||
.saveAsTable("mapInParquet") // This one internally calls df2.insertInto.
|
.saveAsTable("mapInParquet") // This one internally calls df2.insertInto.
|
||||||
createDataFrame(Tuple1(Map(6 -> null.asInstanceOf[Integer])) :: Nil).toDF("a").write
|
|
||||||
.format("parquet").mode(SaveMode.Append).saveAsTable("mapInParquet")
|
(Tuple1(Map(6 -> null.asInstanceOf[Integer])) :: Nil).toDF("a")
|
||||||
|
.write
|
||||||
|
.format("parquet")
|
||||||
|
.mode(SaveMode.Append)
|
||||||
|
.saveAsTable("mapInParquet")
|
||||||
|
|
||||||
refreshTable("mapInParquet")
|
refreshTable("mapInParquet")
|
||||||
|
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
|
@ -655,17 +704,15 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
Row(Map(2 -> 3)) ::
|
Row(Map(2 -> 3)) ::
|
||||||
Row(Map(4 -> 5)) ::
|
Row(Map(4 -> 5)) ::
|
||||||
Row(Map(6 -> null)) :: Nil)
|
Row(Map(6 -> null)) :: Nil)
|
||||||
|
}
|
||||||
sql("DROP TABLE mapInParquet")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
test("SPARK-6024 wide schema support") {
|
test("SPARK-6024 wide schema support") {
|
||||||
|
withSQLConf(SQLConf.SCHEMA_STRING_LENGTH_THRESHOLD -> "4000") {
|
||||||
|
withTable("wide_schema") {
|
||||||
// We will need 80 splits for this schema if the threshold is 4000.
|
// We will need 80 splits for this schema if the threshold is 4000.
|
||||||
val schema = StructType((1 to 5000).map(i => StructField(s"c_${i}", StringType, true)))
|
val schema = StructType((1 to 5000).map(i => StructField(s"c_$i", StringType, true)))
|
||||||
assert(
|
|
||||||
schema.json.size > conf.schemaStringLengthThreshold,
|
|
||||||
"To correctly test the fix of SPARK-6024, the value of " +
|
|
||||||
s"spark.sql.sources.schemaStringLengthThreshold needs to be less than ${schema.json.size}")
|
|
||||||
// Manually create a metastore data source table.
|
// Manually create a metastore data source table.
|
||||||
catalog.createDataSourceTable(
|
catalog.createDataSourceTable(
|
||||||
tableName = "wide_schema",
|
tableName = "wide_schema",
|
||||||
|
@ -680,11 +727,13 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
val actualSchema = table("wide_schema").schema
|
val actualSchema = table("wide_schema").schema
|
||||||
assert(schema === actualSchema)
|
assert(schema === actualSchema)
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") {
|
test("SPARK-6655 still support a schema stored in spark.sql.sources.schema") {
|
||||||
val tableName = "spark6655"
|
val tableName = "spark6655"
|
||||||
|
withTable(tableName) {
|
||||||
val schema = StructType(StructField("int", IntegerType, true) :: Nil)
|
val schema = StructType(StructField("int", IntegerType, true) :: Nil)
|
||||||
|
|
||||||
val hiveTable = HiveTable(
|
val hiveTable = HiveTable(
|
||||||
specifiedDatabase = Some("default"),
|
specifiedDatabase = Some("default"),
|
||||||
name = tableName,
|
name = tableName,
|
||||||
|
@ -703,21 +752,18 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
invalidateTable(tableName)
|
invalidateTable(tableName)
|
||||||
val actualSchema = table(tableName).schema
|
val actualSchema = table(tableName).schema
|
||||||
assert(schema === actualSchema)
|
assert(schema === actualSchema)
|
||||||
sql(s"drop table $tableName")
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
test("Saving partition columns information") {
|
test("Saving partition columns information") {
|
||||||
val df =
|
val df = (1 to 10).map(i => (i, i + 1, s"str$i", s"str${i + 1}")).toDF("a", "b", "c", "d")
|
||||||
sparkContext.parallelize(1 to 10, 4).map { i =>
|
|
||||||
Tuple4(i, i + 1, s"str$i", s"str${i + 1}")
|
|
||||||
}.toDF("a", "b", "c", "d")
|
|
||||||
|
|
||||||
val tableName = s"partitionInfo_${System.currentTimeMillis()}"
|
val tableName = s"partitionInfo_${System.currentTimeMillis()}"
|
||||||
|
|
||||||
|
withTable(tableName) {
|
||||||
df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName)
|
df.write.format("parquet").partitionBy("d", "b").saveAsTable(tableName)
|
||||||
invalidateTable(tableName)
|
invalidateTable(tableName)
|
||||||
val metastoreTable = catalog.client.getTable("default", tableName)
|
val metastoreTable = catalog.client.getTable("default", tableName)
|
||||||
val expectedPartitionColumns =
|
val expectedPartitionColumns = StructType(df.schema("d") :: df.schema("b") :: Nil)
|
||||||
StructType(df.schema("d") :: df.schema("b") :: Nil)
|
|
||||||
val actualPartitionColumns =
|
val actualPartitionColumns =
|
||||||
StructType(
|
StructType(
|
||||||
metastoreTable.partitionColumns.map(c =>
|
metastoreTable.partitionColumns.map(c =>
|
||||||
|
@ -730,16 +776,17 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
|
|
||||||
// Check the content of the saved table.
|
// Check the content of the saved table.
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
table(tableName).selectExpr("c", "b", "d", "a"),
|
table(tableName).select("c", "b", "d", "a"),
|
||||||
df.selectExpr("c", "b", "d", "a").collect())
|
df.select("c", "b", "d", "a"))
|
||||||
|
}
|
||||||
sql(s"drop table $tableName")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
test("insert into a table") {
|
test("insert into a table") {
|
||||||
def createDF(from: Int, to: Int): DataFrame =
|
def createDF(from: Int, to: Int): DataFrame = {
|
||||||
createDataFrame((from to to).map(i => Tuple2(i, s"str$i"))).toDF("c1", "c2")
|
(from to to).map(i => i -> s"str$i").toDF("c1", "c2")
|
||||||
|
}
|
||||||
|
|
||||||
|
withTable("insertParquet") {
|
||||||
createDF(0, 9).write.format("parquet").saveAsTable("insertParquet")
|
createDF(0, 9).write.format("parquet").saveAsTable("insertParquet")
|
||||||
checkAnswer(
|
checkAnswer(
|
||||||
sql("SELECT p.c1, p.c2 FROM insertParquet p WHERE p.c1 > 5"),
|
sql("SELECT p.c1, p.c2 FROM insertParquet p WHERE p.c1 > 5"),
|
||||||
|
@ -788,3 +835,4 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
|
||||||
(70 to 79).map(i => Row(i, s"str$i")))
|
(70 to 79).map(i => Row(i, s"str$i")))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
Loading…
Reference in a new issue