Skip to content
Closed
Changes from 1 commit
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
Prev Previous commit
Next Next commit
modfiy test case
  • Loading branch information
windpiger committed Feb 28, 2017
commit 4f660d2211d597aa64105f925b981d081449c58d
Original file line number Diff line number Diff line change
Expand Up @@ -1496,19 +1496,19 @@ class HiveDDLSuite
}
}

test("insert data to a hive serde table which has a not existed location should succeed") {
test("insert data to a hive serde table which has a non-existing location should succeed") {
withTable("t") {
withTempDir { dir =>
val dirPath = dir.getAbsolutePath.stripSuffix("/")
spark.sql(
s"""
|CREATE TABLE t(a string, b int)
|USING hive
|OPTIONS(path "file:$dirPath")
|LOCATION '$dir'
Copy link
Contributor

Choose a reason for hiding this comment

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

can we just call dir.delete before creating this table?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok~

Copy link
Contributor Author

@windpiger windpiger Mar 2, 2017

Choose a reason for hiding this comment

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

@cloud-fan I found the dir will be created in create table, so we should keep current logic.

Copy link
Contributor

Choose a reason for hiding this comment

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

does hive have the same behavior?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yes, I test it in Hive

create table test(a string) location 'hdfs:/xx';

then hdfs:/xx will be created

Copy link
Contributor

Choose a reason for hiding this comment

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

seems the InMemoryCatalog doesn't do this, you can send a new PR to fix it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok thanks~

""".stripMargin)
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
val expectedPath = s"file:$dirPath"
assert(table.location.stripSuffix("/") == expectedPath)
val dirPath = new Path(dir.getAbsolutePath)
val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf())
assert(new Path(table.location) == fs.makeQualified(dirPath))

val tableLocFile = new File(table.location.stripPrefix("file:"))
Copy link
Contributor

Choose a reason for hiding this comment

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

new File(new URI(table.location))? please avoid .stripPrefix("file:") which looks very hacky.

tableLocFile.delete()
Expand Down Expand Up @@ -1539,20 +1539,20 @@ class HiveDDLSuite
}
}

test("insert into a hive serde table with no existed partition location should succeed") {
test("insert into a hive serde table with non-existing partition location should succeed") {
withTable("t") {
withTempDir { dir =>
val dirPath = dir.getAbsolutePath.stripSuffix("/")
spark.sql(
s"""
|CREATE TABLE t(a int, b int, c int, d int)
|USING hive
|PARTITIONED BY(a, b)
|LOCATION "file:$dirPath"
|LOCATION "$dir"
""".stripMargin)
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
val expectedPath = s"file:$dirPath"
assert(table.location.stripSuffix("/") == expectedPath)
val dirPath = new Path(dir.getAbsolutePath)
val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf())
assert(new Path(table.location) == fs.makeQualified(dirPath))

spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4")
checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil)
Expand All @@ -1578,19 +1578,19 @@ class HiveDDLSuite
}
}

test("read data from a hive serde table which has a not existed location should succeed") {
test("read data from a hive serde table which has a non-existing location should succeed") {
withTable("t") {
withTempDir { dir =>
val dirPath = dir.getAbsolutePath.stripSuffix("/")
spark.sql(
s"""
|CREATE TABLE t(a string, b int)
|USING hive
|OPTIONS(path "file:${dir.getAbsolutePath}")
|LOCATION "$dir"
""".stripMargin)
val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t"))
val expectedPath = s"file:$dirPath"
assert(table.location.stripSuffix("/") == expectedPath)
val dirPath = new Path(dir.getAbsolutePath)
val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf())
assert(new Path(table.location) == fs.makeQualified(dirPath))

dir.delete()
checkAnswer(spark.table("t"), Nil)
Expand All @@ -1607,7 +1607,7 @@ class HiveDDLSuite
}
}

test("read data from a hive serde table with no existed partition location should succeed") {
test("read data from a hive serde table with non-existing partition location should succeed") {
withTable("t") {
withTempDir { dir =>
spark.sql(
Expand Down