diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala index cc25318c6a4b4..1025a89f653b8 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/RepairHoodieTableCommand.scala @@ -67,8 +67,7 @@ case class RepairHoodieTableCommand(tableName: TableIdentifier, logInfo(s"Recover all the partitions in $root") val hoodieCatalogTable = HoodieCatalogTable(spark, table.identifier) - val isHiveStyledPartitioning = hoodieCatalogTable.catalogProperties. - getOrElse(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key, "true").toBoolean + val isHiveStyledPartitioning = hoodieCatalogTable.tableConfig.getHiveStylePartitioningEnable.toBoolean val partitionSpecsAndLocs: Seq[(TablePartitionSpec, Path)] = hoodieCatalogTable. getPartitionPaths.map(partitionPath => { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala index 498121e1ab4a3..10c2b443c01ce 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/TestRepairTable.scala @@ -98,6 +98,40 @@ class TestRepairTable extends HoodieSparkSqlTestBase { } } + test("Test msck repair external partitioned table") { + Seq("true", "false").foreach { hiveStylePartitionEnable => + withTempDir { tmp => + val tableName = generateTableName + val basePath = s"${tmp.getCanonicalPath}/$tableName" + + import spark.implicits._ + val df = Seq((1, "a1", 1000, "2022-10-06", "11"), (2, "a2", 1001, "2022-10-06", "12")) + .toDF("id", "name", "ts", "dt", "hh") + df.write.format("hudi") + .option(TBL_NAME.key(), tableName) + .option(RECORDKEY_FIELD.key, "id") + .option(PRECOMBINE_FIELD.key, "ts") + .option(PARTITIONPATH_FIELD.key, "dt, hh") + .option(HIVE_STYLE_PARTITIONING_ENABLE.key, hiveStylePartitionEnable) + .mode(SaveMode.Append) + .save(basePath) + + spark.sql( + s""" + | create table $tableName + | using hudi + | location '$basePath' + """.stripMargin) + val table = spark.sessionState.sqlParser.parseTableIdentifier(tableName) + + assertResult(Seq())(spark.sessionState.catalog.listPartitionNames(table)) + spark.sql(s"msck repair table $tableName") + assertResult(Seq("dt=2022-10-06/hh=11", "dt=2022-10-06/hh=12"))( + spark.sessionState.catalog.listPartitionNames(table)) + } + } + } + test("Test msck repair partitioned table [add/drop/sync] partitions") { if (HoodieSparkUtils.gteqSpark3_2) { Seq("true", "false").foreach { hiveStylePartitionEnable =>