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 f5d08ec1bbae fix: Fix SHOW PARTITIONS commands functionality for
slash-separated date partitioning (#18195)
f5d08ec1bbae is described below
commit f5d08ec1bbaea5145cb58a29f116de783ae8f08a
Author: Surya Prasanna <[email protected]>
AuthorDate: Mon Feb 23 18:31:34 2026 -0800
fix: Fix SHOW PARTITIONS commands functionality for slash-separated date
partitioning (#18195)
---
.../command/ShowHoodieTablePartitionsCommand.scala | 29 ++++++++-----
.../spark/sql/hudi/ddl/TestShowPartitions.scala | 47 ++++++++++++++++++++++
2 files changed, 65 insertions(+), 11 deletions(-)
diff --git
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
index f43b10fcafc4..9b2554fd41b2 100644
---
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
+++
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/ShowHoodieTablePartitionsCommand.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.hudi.command
-import org.apache.hudi.common.util.PartitionPathEncodeUtils
+import org.apache.hudi.common.util.{PartitionPathEncodeUtils, ValidationUtils}
import org.apache.spark.sql.{Row, SparkSession}
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -44,18 +44,25 @@ case class ShowHoodieTablePartitionsCommand(
val schemaOpt = hoodieCatalogTable.tableSchema
val partitionColumnNamesOpt =
hoodieCatalogTable.tableConfig.getPartitionFields
+ val useSlashSeparatedDatePartitioning =
hoodieCatalogTable.tableConfig.getSlashSeparatedDatePartitioning
- if (partitionColumnNamesOpt.isPresent &&
partitionColumnNamesOpt.get.nonEmpty && schemaOpt.nonEmpty) {
- specOpt.map { spec =>
- hoodieCatalogTable.getPartitionPaths.filter { partitionPath =>
- val part = PartitioningUtils.parsePathFragment(partitionPath)
- spec.forall { case (col, value) =>
- PartitionPathEncodeUtils.escapePartitionValue(value) ==
part.getOrElse(col, null)
+ if (partitionColumnNamesOpt.isPresent &&
partitionColumnNamesOpt.get.nonEmpty && schemaOpt.nonEmpty) {
+ specOpt.map { spec =>
+ hoodieCatalogTable.getPartitionPaths.filter { partitionPath =>
+ val part = PartitioningUtils.parsePathFragment(partitionPath)
+ spec.forall { case (col, value) =>
+ PartitionPathEncodeUtils.escapePartitionValue(value) ==
part.getOrElse(col, null)
+ }
+ }
}
- }
- }
- .getOrElse(hoodieCatalogTable.getPartitionPaths)
- .map(Row(_))
+ .getOrElse(hoodieCatalogTable.getPartitionPaths)
+ .map(partitionVal => if (useSlashSeparatedDatePartitioning) {
+ ValidationUtils.checkState(partitionColumnNamesOpt.get().length ==
1,
+ "Only one partition field is allowed for
SlashEncodedPartitioning")
+ Row(partitionColumnNamesOpt.get()(0) + "=" +
partitionVal.replace('/', '-'))
+ } else {
+ Row(partitionVal)
+ })
} else {
Seq.empty[Row]
}
diff --git
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
index cf9ddb4f9b3e..6ac471568a73 100644
---
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
+++
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestShowPartitions.scala
@@ -305,4 +305,51 @@ class TestShowPartitions extends HoodieSparkSqlTestBase {
}
}
}
+
+ test("Test show partitions with slash separated list partitions") {
+ withTempDir { tmp =>
+ val tableName = generateTableName
+ val tablePath = s"${tmp.getCanonicalPath}/$tableName"
+
+ // Create a table with slash separated date partitioning enabled
+ spark.sql(
+ s"""
+ | create table $tableName (
+ | id int,
+ | name string,
+ | price double,
+ | ts long,
+ | dt string
+ | ) using hudi
+ | partitioned by (dt)
+ | tblproperties (
+ | primaryKey = 'id',
+ | orderingFields = 'ts',
+ | 'hoodie.datasource.write.slash.separated.date.partitioning' =
'true'
+ | )
+ | location '$tablePath'
+ """.stripMargin)
+
+ // Empty partitions
+ checkAnswer(s"show partitions $tableName")(Seq.empty: _*)
+
+ // Insert into dynamic partition with date values
+ spark.sql(
+ s"""
+ | insert into $tableName
+ | values
+ | (1, 'a1', 10, 1000, '2026-01-05'),
+ | (2, 'a2', 10, 1000, '2026-01-06'),
+ | (3, 'a3', 10, 1000, '2026-02-10')
+ """.stripMargin)
+
+ // Check partitions are listed correctly
+ // With slash-separated partitioning enabled, SHOW PARTITIONS returns
physical paths
+ checkAnswer(s"show partitions $tableName")(
+ Seq("dt=2026-01-05"),
+ Seq("dt=2026-01-06"),
+ Seq("dt=2026-02-10")
+ )
+ }
+ }
}