Github user windpiger commented on a diff in the pull request: https://github.com/apache/spark/pull/16910#discussion_r104309883 --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala --- @@ -1588,6 +1596,147 @@ class HiveDDLSuite } } + test("insert data to a hive serde table which has a non-existing location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, b int) + |USING hive + |LOCATION '$dir' + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(new Path(table.location) == makeQualifiedPath(dir.getAbsolutePath)) + + val tableLocFile = new File(new URI(table.location)) + tableLocFile.delete() + assert(!tableLocFile.exists()) + spark.sql("INSERT INTO TABLE t SELECT 'c', 1") + assert(tableLocFile.exists()) + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + + Utils.deleteRecursively(dir) + assert(!tableLocFile.exists()) + spark.sql("INSERT OVERWRITE TABLE t SELECT 'c', 1") + assert(tableLocFile.exists()) + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + + val newDirFile = new File(dir, "x") + val newDirPath = newDirFile.getAbsolutePath + spark.sql(s"ALTER TABLE t SET LOCATION '$newDirPath'") + spark.sessionState.catalog.refreshTable(TableIdentifier("t")) + + val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table1.location == newDirPath) + assert(!newDirFile.exists()) + + spark.sql("INSERT INTO TABLE t SELECT 'c', 1") + checkAnswer(spark.table("t"), Row("c", 1) :: Nil) + assert(newDirFile.exists()) + } + } + } + + test("insert into a hive serde table with non-existing partition location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a int, b int, c int, d int) + |USING hive + |PARTITIONED BY(a, b) + |LOCATION "$dir" + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(new Path(table.location) == makeQualifiedPath(dir.getAbsolutePath)) + + spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4") + checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil) + + val partLoc = new File(dir, "a=1") + Utils.deleteRecursively(partLoc) + assert(!partLoc.exists()) + // insert overwrite into a partition which location has been deleted. + spark.sql("INSERT OVERWRITE TABLE t PARTITION(a=1, b=2) SELECT 7, 8") + assert(partLoc.exists()) + checkAnswer(spark.table("t"), Row(7, 8, 1, 2) :: Nil) + + val newDirFile = new File(dir, "x") + val newDirPath = newDirFile.getAbsolutePath + spark.sql(s"ALTER TABLE t PARTITION(a=1, b=2) SET LOCATION '$newDirPath'") + assert(!newDirFile.exists()) + + // insert into a partition which location does not exists. + spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 9, 10") + assert(newDirFile.exists()) + checkAnswer(spark.table("t"), Row(9, 10, 1, 2) :: Nil) + } + } + } + + test("read data from a hive serde table which has a non-existing location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a string, b int) + |USING hive + |LOCATION "$dir" + """.stripMargin) + val table = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(new Path(table.location) == makeQualifiedPath(dir.getAbsolutePath)) + + dir.delete() + checkAnswer(spark.table("t"), Nil) + + val newDirFile = new File(dir, "x") + val newDirPath = newDirFile.getAbsolutePath + spark.sql(s"ALTER TABLE t SET LOCATION '$newDirPath'") + + val table1 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + assert(table1.location == newDirPath) + assert(!newDirFile.exists()) + checkAnswer(spark.table("t"), Nil) + } + } + } + + test("read data from a hive serde table with non-existing partition location should succeed") { + withTable("t") { + withTempDir { dir => + spark.sql( + s""" + |CREATE TABLE t(a int, b int, c int, d int) + |USING hive + |PARTITIONED BY(a, b) + |LOCATION "$dir" + """.stripMargin) + spark.sql("INSERT INTO TABLE t PARTITION(a=1, b=2) SELECT 3, 4") + checkAnswer(spark.table("t"), Row(3, 4, 1, 2) :: Nil) + + val newDirFile = new File(dir, "x") + val newDirPath = newDirFile.getAbsolutePath + spark.sql(s"ALTER TABLE t PARTITION(a=1, b=2) SET LOCATION '$newDirPath'") + assert(!newDirFile.exists()) + // select from a partition which location has changed to a not existed location + withSQLConf(SQLConf.HIVE_VERIFY_PARTITION_PATH.key -> "true") { --- End diff -- Yes, hive return empty , if there is a bug here(could you describe what the bug is?), we can remove the conf ,and always return result?
--- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org