This is an automated email from the ASF dual-hosted git repository.
danny0405 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git
The following commit(s) were added to refs/heads/master by this push:
new c7320f78407 [HUDI-6998] Fix drop table failure when load table as
spark v2 table whose path is delete (#9932)
c7320f78407 is described below
commit c7320f78407a68c009954b74faade03dd4fb494c
Author: Wechar Yu <[email protected]>
AuthorDate: Tue Oct 31 12:53:53 2023 +0800
[HUDI-6998] Fix drop table failure when load table as spark v2 table whose
path is delete (#9932)
---
.../sql/catalyst/catalog/HoodieCatalogTable.scala | 7 ++-
.../org/apache/spark/sql/hudi/TestDropTable.scala | 53 ++++++++++++----------
2 files changed, 35 insertions(+), 25 deletions(-)
diff --git
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
index 01fa4f7e39b..df47f63a8c0 100644
---
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
+++
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala
@@ -319,7 +319,7 @@ class HoodieCatalogTable(val spark: SparkSession, var
table: CatalogTable) exten
private def loadTableSchemaByMetaClient(): Option[StructType] = {
val resolver = spark.sessionState.conf.resolver
- getTableSqlSchema(metaClient, includeMetadataFields =
true).map(originSchema => {
+ try getTableSqlSchema(metaClient, includeMetadataFields =
true).map(originSchema => {
// Load table schema from meta on filesystem, and fill in 'comment'
// information from Spark catalog.
// Hoodie newly added columns are positioned after partition columns,
@@ -335,6 +335,11 @@ class HoodieCatalogTable(val spark: SparkSession, var
table: CatalogTable) exten
}.partition(f => partitionFields.contains(f.name))
StructType(dataFields ++ partFields)
})
+ catch {
+ case cause: Throwable =>
+ logWarning("Failed to load table schema from meta client.", cause)
+ None
+ }
}
// This code is forked from
org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema
diff --git
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala
index b86241eaca9..3f5dc3a1d64 100644
---
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala
+++
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestDropTable.scala
@@ -325,30 +325,35 @@ class TestDropTable extends HoodieSparkSqlTestBase {
}
}
- test("Drop an MANAGED table which path is lost.") {
- val tableName = generateTableName
- spark.sql(
- s"""
- |create table $tableName (
- |id int,
- |ts int,
- |value string
- |)using hudi
- | tblproperties (
- | primaryKey = 'id',
- | preCombineField = 'ts'
- | )
- |""".stripMargin)
-
- val tablePath = new Path(
-
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location)
-
- val filesystem = FSUtils.getFs(tablePath,
spark.sparkContext.hadoopConfiguration);
- assert(filesystem.exists(tablePath), s"Table path doesn't exists
($tablePath).")
-
- filesystem.delete(tablePath, true)
- spark.sql(s"drop table ${tableName}")
- checkAnswer("show tables")()
+ test("Drop a MANAGED table whose path is lost when schema evolution is
applied/unapplied.") {
+ Seq("true", "false").foreach { enableSchemaEvolution =>
+ withSQLConf("hoodie.schema.on.read.enable" -> enableSchemaEvolution) {
+ withTable(generateTableName) { tableName =>
+ spark.sql(
+ s"""
+ |create table $tableName (
+ |id int,
+ |ts int,
+ |value string
+ |)using hudi
+ | tblproperties (
+ | primaryKey = 'id',
+ | preCombineField = 'ts'
+ | )
+ |""".stripMargin)
+
+ val tablePath = new Path(
+
spark.sessionState.catalog.getTableMetadata(TableIdentifier(tableName)).location)
+
+ val filesystem = FSUtils.getFs(tablePath,
spark.sparkContext.hadoopConfiguration);
+ assert(filesystem.exists(tablePath), s"Table path doesn't exists
($tablePath).")
+
+ filesystem.delete(tablePath, true)
+ spark.sql(s"drop table $tableName")
+ checkAnswer("show tables")()
+ }
+ }
+ }
}
test("Drop local temporary view should not fail") {