This is an automated email from the ASF dual-hosted git repository.

sivabalan 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 983c03d098e3 feat: Support slash separated date partitioning for Hudi 
tables (#17787)
983c03d098e3 is described below

commit 983c03d098e39efaeba690c4508813f632df4bdb
Author: Surya Prasanna <[email protected]>
AuthorDate: Fri Jan 30 10:01:18 2026 -0800

    feat: Support slash separated date partitioning for Hudi tables (#17787)
---
 .../hudi/keygen/ComplexAvroKeyGenerator.java       |   3 +-
 .../java/org/apache/hudi/keygen/KeyGenUtils.java   |  20 ++-
 .../apache/hudi/keygen/SimpleAvroKeyGenerator.java |   3 +-
 .../hudi/client/utils/SparkPartitionUtils.java     |   8 +-
 .../apache/hudi/keygen/BuiltinKeyGenerator.java    |   6 +-
 .../hudi/keygen/PartitionPathFormatterBase.java    |  14 +-
 .../hudi/keygen/StringPartitionPathFormatter.java  |   7 +-
 .../keygen/UTF8StringPartitionPathFormatter.java   |   7 +-
 .../hudi/table/HoodieSparkCopyOnWriteTable.java    |   4 +-
 .../commit/BaseSparkCommitActionExecutor.java      |   4 +-
 .../scala/org/apache/hudi/HoodieSparkUtils.scala   |  33 +++--
 .../hudi/keygen/TestComplexKeyGenerator.java       |  40 +++++
 .../apache/hudi/keygen/TestCustomKeyGenerator.java |  44 ++++++
 .../apache/hudi/keygen/TestSimpleKeyGenerator.java |  45 ++++++
 .../hudi/common/table/HoodieTableConfig.java       |   5 +
 .../hudi/common/table/HoodieTableMetaClient.java   |  12 ++
 .../org/apache/hudi/keygen/BaseKeyGenerator.java   |   3 +
 .../hudi/keygen/constant/KeyGeneratorOptions.java  |   8 +
 .../hudi/common/table/TestHoodieTableConfig.java   |   2 +-
 .../apache/hudi/SparkHoodieTableFileIndex.scala    |   4 +-
 .../sql/catalyst/catalog/HoodieCatalogTable.scala  |  16 +-
 .../spark/sql/hudi/HoodieSqlCommonUtils.scala      |  13 ++
 .../hudi/command/CreateHoodieTableCommand.scala    |   2 +
 .../spark/sql/hudi/command/SqlKeyGenerator.scala   |   8 +-
 .../common/TestSlashSeparatedPartitionValue.scala  | 161 +++++++++++++++++++++
 .../apache/hudi/sync/common/HoodieSyncConfig.java  |   4 +
 26 files changed, 434 insertions(+), 42 deletions(-)

diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
index f2e8b8b36e6b..19ff84b60886 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
@@ -50,7 +50,8 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator 
{
 
   @Override
   public String getPartitionPath(GenericRecord record) {
-    return KeyGenUtils.getRecordPartitionPath(record, 
getPartitionPathFields(), hiveStylePartitioning, encodePartitionPath, 
isConsistentLogicalTimestampEnabled());
+    return KeyGenUtils.getRecordPartitionPath(record, 
getPartitionPathFields(), hiveStylePartitioning,
+        encodePartitionPath, slashSeparatedDatePartitioning, 
isConsistentLogicalTimestampEnabled());
   }
 
   private SerializableFunctionUnchecked<GenericRecord, String> 
getRecordKeyFunc(boolean encodeSingleKeyFieldName) {
diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
index 6cd7d193b78a..c2c90138082f 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
@@ -233,8 +233,12 @@ public class KeyGenUtils {
     return constructRecordKey(recordKeyFields.toArray(new String[]{}), 
valueFunction);
   }
 
-  public static String getRecordPartitionPath(GenericRecord record, 
List<String> partitionPathFields,
-                                              boolean hiveStylePartitioning, 
boolean encodePartitionPath, boolean consistentLogicalTimestampEnabled) {
+  public static String getRecordPartitionPath(GenericRecord record,
+                                              List<String> partitionPathFields,
+                                              boolean hiveStylePartitioning,
+                                              boolean encodePartitionPath,
+                                              boolean 
slashSeparatedDatePartitioning,
+                                              boolean 
consistentLogicalTimestampEnabled) {
     if (partitionPathFields.isEmpty()) {
       return "";
     }
@@ -253,7 +257,10 @@ public class KeyGenUtils {
           fieldVal = PartitionPathEncodeUtils.escapePathName(fieldVal);
         }
         if (hiveStylePartitioning) {
-          partitionPath.append(partitionPathField).append("=");
+          fieldVal = partitionPathField + "=" + fieldVal;
+        }
+        if (partitionPathFields.size() == 1 && slashSeparatedDatePartitioning) 
{
+          fieldVal = fieldVal.replace('-', '/');
         }
         partitionPath.append(fieldVal);
       }
@@ -273,7 +280,9 @@ public class KeyGenUtils {
   }
 
   public static String getPartitionPath(GenericRecord record, String 
partitionPathField,
-                                        boolean hiveStylePartitioning, boolean 
encodePartitionPath, boolean consistentLogicalTimestampEnabled) {
+                                        boolean hiveStylePartitioning, boolean 
encodePartitionPath,
+                                        boolean slashSeparatedDatePartitioning,
+                                        boolean 
consistentLogicalTimestampEnabled) {
     String partitionPath = HoodieAvroUtils.getNestedFieldValAsString(record, 
partitionPathField, true, consistentLogicalTimestampEnabled);
     if (partitionPath == null || partitionPath.isEmpty()) {
       partitionPath = HUDI_DEFAULT_PARTITION_PATH;
@@ -284,6 +293,9 @@ public class KeyGenUtils {
     if (hiveStylePartitioning) {
       partitionPath = partitionPathField + "=" + partitionPath;
     }
+    if (slashSeparatedDatePartitioning) {
+      partitionPath = partitionPath.replace('-', '/');
+    }
     return partitionPath;
   }
 
diff --git 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
index 5546d821adf0..5f9934fc78b4 100644
--- 
a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
+++ 
b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
@@ -52,6 +52,7 @@ public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   @Override
   public String getPartitionPath(GenericRecord record) {
-    return KeyGenUtils.getPartitionPath(record, 
getPartitionPathFields().get(0), hiveStylePartitioning, encodePartitionPath, 
isConsistentLogicalTimestampEnabled());
+    return KeyGenUtils.getPartitionPath(record, 
getPartitionPathFields().get(0), hiveStylePartitioning,
+        encodePartitionPath, slashSeparatedDatePartitioning,  
isConsistentLogicalTimestampEnabled());
   }
 }
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkPartitionUtils.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkPartitionUtils.java
index d3f7f9908324..1bfce2b30c50 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkPartitionUtils.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkPartitionUtils.java
@@ -21,6 +21,7 @@ package org.apache.hudi.client.utils;
 import org.apache.hudi.HoodieSchemaConversionUtils;
 import org.apache.hudi.HoodieSparkUtils;
 import org.apache.hudi.common.schema.HoodieSchema;
+import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.storage.StoragePath;
 
@@ -29,20 +30,23 @@ import org.apache.spark.sql.internal.SQLConf;
 
 public class SparkPartitionUtils {
 
-  public static Object[] getPartitionFieldVals(Option<String[]> 
partitionFields,
+  public static Object[] getPartitionFieldVals(HoodieTableConfig tableConfig,
                                                String partitionPath,
                                                String basePath,
                                                HoodieSchema writerSchema,
                                                Configuration hadoopConf) {
+    Option<String[]> partitionFields = tableConfig.getPartitionFields();
     if (!partitionFields.isPresent()) {
       return new Object[0];
     }
+    boolean slashSeparatedDatePartitioning = 
tableConfig.getSlashSeparatedDatePartitioning();
     return HoodieSparkUtils.doParsePartitionColumnValues(
         partitionFields.get(),
         partitionPath,
         new StoragePath(basePath),
         
HoodieSchemaConversionUtils.convertHoodieSchemaToStructType(writerSchema),
         hadoopConf.get("timeZone", SQLConf.get().sessionLocalTimeZone()),
-        hadoopConf.getBoolean("spark.sql.sources.validatePartitionColumns", 
true));
+        hadoopConf.getBoolean("spark.sql.sources.validatePartitionColumns", 
true),
+        slashSeparatedDatePartitioning);
   }
 }
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
index bb757ba04176..76c2c7571c04 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
@@ -298,7 +298,8 @@ public abstract class BuiltinKeyGenerator extends 
BaseKeyGenerator implements Sp
       synchronized (this) {
         if (stringPartitionPathFormatter == null) {
           this.stringPartitionPathFormatter = new StringPartitionPathFormatter(
-              StringPartitionPathFormatter.JavaStringBuilder::new, 
hiveStylePartitioning, encodePartitionPath);
+              StringPartitionPathFormatter.JavaStringBuilder::new, 
hiveStylePartitioning,
+              encodePartitionPath, slashSeparatedDatePartitioning);
         }
       }
     }
@@ -311,7 +312,8 @@ public abstract class BuiltinKeyGenerator extends 
BaseKeyGenerator implements Sp
       synchronized (this) {
         if (utf8StringPartitionPathFormatter == null) {
           this.utf8StringPartitionPathFormatter = new 
UTF8StringPartitionPathFormatter(
-              UTF8StringPartitionPathFormatter.UTF8StringBuilder::new, 
hiveStylePartitioning, encodePartitionPath);
+              UTF8StringPartitionPathFormatter.UTF8StringBuilder::new, 
hiveStylePartitioning,
+              encodePartitionPath, slashSeparatedDatePartitioning);
         }
       }
     }
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/PartitionPathFormatterBase.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/PartitionPathFormatterBase.java
index 37136e4e625d..d08545cf8fd5 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/PartitionPathFormatterBase.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/PartitionPathFormatterBase.java
@@ -44,14 +44,17 @@ public abstract class PartitionPathFormatterBase<S> {
 
   private final boolean useHiveStylePartitioning;
   private final boolean useEncoding;
+  private final boolean slashSeparatedDatePartitioning;
 
   PartitionPathFormatterBase(Supplier<StringBuilder<S>> stringBuilderFactory,
                              boolean useHiveStylePartitioning,
-                             boolean useEncoding) {
+                             boolean useEncoding,
+                             boolean slashSeparatedDatePartitioning) {
     this.stringBuilderFactory = stringBuilderFactory;
 
     this.useHiveStylePartitioning = useHiveStylePartitioning;
     this.useEncoding = useEncoding;
+    this.slashSeparatedDatePartitioning = slashSeparatedDatePartitioning;
   }
 
   public final S combine(List<String> partitionPathFields, Object... 
partitionPathParts) {
@@ -59,7 +62,11 @@ public abstract class PartitionPathFormatterBase<S> {
     // Avoid creating [[StringBuilder]] in case there's just one 
partition-path part,
     // and Hive-style of partitioning is not required
     if (!useHiveStylePartitioning && partitionPathParts.length == 1) {
-      return tryEncode(handleEmpty(toString(partitionPathParts[0])));
+      if (slashSeparatedDatePartitioning) {
+        return ((S) ((String) toString(partitionPathParts[0])).replace('-', 
'/'));
+      } else {
+        return tryEncode(handleEmpty(toString(partitionPathParts[0])));
+      }
     }
 
     StringBuilder<S> sb = stringBuilderFactory.get();
@@ -70,6 +77,9 @@ public abstract class PartitionPathFormatterBase<S> {
         sb.appendJava(partitionPathFields.get(i))
             .appendJava("=")
             .append(partitionPathPartStr);
+      } else if (slashSeparatedDatePartitioning) {
+        String res = ((String) partitionPathPartStr).replace('-', '/');
+        sb.append(((S) res));
       } else {
         sb.append(partitionPathPartStr);
       }
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/StringPartitionPathFormatter.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/StringPartitionPathFormatter.java
index 7fd7fe6ff5a7..5709193b5fdf 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/StringPartitionPathFormatter.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/StringPartitionPathFormatter.java
@@ -29,9 +29,10 @@ import static 
org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
 public class StringPartitionPathFormatter extends 
PartitionPathFormatterBase<String> {
 
   public StringPartitionPathFormatter(Supplier<StringBuilder<String>> 
stringBuilderFactory,
-                               boolean useHiveStylePartitioning,
-                               boolean useEncoding) {
-    super(stringBuilderFactory, useHiveStylePartitioning, useEncoding);
+                                      boolean useHiveStylePartitioning,
+                                      boolean useEncoding,
+                                      boolean slashSeparatedDatePartitioning) {
+    super(stringBuilderFactory, useHiveStylePartitioning, useEncoding, 
slashSeparatedDatePartitioning);
   }
 
   @Override
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java
index dec07f34eaa6..f9da94ce8bc0 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/UTF8StringPartitionPathFormatter.java
@@ -34,9 +34,10 @@ public class UTF8StringPartitionPathFormatter extends 
PartitionPathFormatterBase
   protected static final UTF8String HUDI_DEFAULT_PARTITION_PATH_UTF8 = 
UTF8String.fromString(HUDI_DEFAULT_PARTITION_PATH);
 
   public UTF8StringPartitionPathFormatter(Supplier<StringBuilder<UTF8String>> 
stringBuilderFactory,
-                                   boolean useHiveStylePartitioning,
-                                   boolean useEncoding) {
-    super(stringBuilderFactory, useHiveStylePartitioning, useEncoding);
+                                          boolean useHiveStylePartitioning,
+                                          boolean useEncoding,
+                                          boolean 
slashSeparatedDatePartitioning) {
+    super(stringBuilderFactory, useHiveStylePartitioning, useEncoding, 
slashSeparatedDatePartitioning);
   }
 
   @Override
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
index ce450a2787f0..7c2be198a61d 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
@@ -258,8 +258,8 @@ public class HoodieSparkCopyOnWriteTable<T>
         dataFileToBeMerged, taskContextSupplier, keyGeneratorOpt);
     if (mergeHandle.getOldFilePath() != null && 
mergeHandle.baseFileForMerge().getBootstrapBaseFile().isPresent()) {
       Option<String[]> partitionFields = 
getMetaClient().getTableConfig().getPartitionFields();
-      Object[] partitionValues = 
SparkPartitionUtils.getPartitionFieldVals(partitionFields, 
mergeHandle.getPartitionPath(),
-          getMetaClient().getTableConfig().getBootstrapBasePath().get(),
+      Object[] partitionValues = 
SparkPartitionUtils.getPartitionFieldVals(getMetaClient().getTableConfig(),
+          mergeHandle.getPartitionPath(), 
getMetaClient().getTableConfig().getBootstrapBasePath().get(),
           mergeHandle.getWriterSchema(), (Configuration) 
getStorageConf().unwrap());
       mergeHandle.setPartitionFields(partitionFields);
       mergeHandle.setPartitionValues(partitionValues);
diff --git 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
index 9e3dba825d7f..e79ee136e98f 100644
--- 
a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
+++ 
b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
@@ -397,8 +397,8 @@ public abstract class BaseSparkCommitActionExecutor<T> 
extends
           taskContextSupplier, keyGeneratorOpt);
     if (mergeHandle.getOldFilePath() != null && 
mergeHandle.baseFileForMerge().getBootstrapBaseFile().isPresent()) {
       Option<String[]> partitionFields = 
table.getMetaClient().getTableConfig().getPartitionFields();
-      Object[] partitionValues = 
SparkPartitionUtils.getPartitionFieldVals(partitionFields, 
mergeHandle.getPartitionPath(),
-          table.getMetaClient().getTableConfig().getBootstrapBasePath().get(),
+      Object[] partitionValues = 
SparkPartitionUtils.getPartitionFieldVals(table.getMetaClient().getTableConfig(),
+          mergeHandle.getPartitionPath(), 
table.getMetaClient().getTableConfig().getBootstrapBasePath().get(),
           mergeHandle.getWriterSchema(), (Configuration) 
table.getStorageConf().unwrap());
       mergeHandle.setPartitionFields(partitionFields);
       mergeHandle.setPartitionValues(partitionValues);
diff --git 
a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
 
b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
index 1a50f5c7d08b..08c315b0e78d 100644
--- 
a/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
+++ 
b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala
@@ -28,9 +28,9 @@ import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator
 import org.apache.hudi.keygen.constant.KeyGeneratorType
 import org.apache.hudi.storage.StoragePath
 import org.apache.hudi.util.ExceptionWrappingIterator
-
 import org.apache.avro.generic.GenericRecord
 import org.apache.hadoop.fs.Path
+import org.apache.hudi.common.table.HoodieTableConfig
 import org.apache.spark.SPARK_VERSION
 import org.apache.spark.internal.Logging
 import org.apache.spark.rdd.RDD
@@ -43,6 +43,8 @@ import org.apache.spark.sql.types.{StringType, StructField, 
StructType}
 import org.apache.spark.sql.DataFrame
 import org.apache.spark.unsafe.types.UTF8String
 
+import java.time.LocalDate
+import java.time.format.{DateTimeFormatter, DateTimeParseException}
 import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
 
@@ -65,6 +67,7 @@ private[hudi] trait SparkVersionsSupport {
 object HoodieSparkUtils extends SparkAdapterSupport with SparkVersionsSupport 
with Logging {
 
   override def getSparkVersion: String = SPARK_VERSION
+  val dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy/MM/dd")
 
   def getMetaSchema: StructType = {
     StructType(HoodieRecord.HOODIE_META_COLUMNS.asScala.map(col => {
@@ -218,12 +221,12 @@ object HoodieSparkUtils extends SparkAdapterSupport with 
SparkVersionsSupport wi
   }
 
   def parsePartitionColumnValues(partitionColumns: Array[String],
-                                   partitionPath: String,
-                                   tableBasePath: StoragePath,
-                                   tableSchema: StructType,
-                                   tableConfig: java.util.Map[String, String],
-                                   timeZoneId: String,
-                                   shouldValidatePartitionColumns: Boolean): 
Array[Object] = {
+                                 partitionPath: String,
+                                 tableBasePath: StoragePath,
+                                 tableSchema: StructType,
+                                 tableConfig: java.util.Map[String, String],
+                                 timeZoneId: String,
+                                 shouldValidatePartitionColumns: Boolean): 
Array[Object] = {
     val keyGeneratorClass = 
KeyGeneratorType.getKeyGeneratorClassName(tableConfig)
     val timestampKeyGeneratorType = 
tableConfig.get(TimestampKeyGeneratorConfig.TIMESTAMP_TYPE_FIELD.key())
 
@@ -237,16 +240,18 @@ object HoodieSparkUtils extends SparkAdapterSupport with 
SparkVersionsSupport wi
       Array.fill(partitionColumns.length)(UTF8String.fromString(partitionPath))
     } else {
       doParsePartitionColumnValues(partitionColumns, partitionPath, 
tableBasePath, tableSchema, timeZoneId,
-        shouldValidatePartitionColumns)
+        shouldValidatePartitionColumns, 
tableConfig.getOrDefault(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key,
+          
HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.defaultValue).toBoolean)
     }
   }
 
   def doParsePartitionColumnValues(partitionColumns: Array[String],
-                                 partitionPath: String,
-                                 basePath: StoragePath,
-                                 schema: StructType,
-                                 timeZoneId: String,
-                                 shouldValidatePartitionCols: Boolean): 
Array[Object] = {
+                                   partitionPath: String,
+                                   basePath: StoragePath,
+                                   schema: StructType,
+                                   timeZoneId: String,
+                                   shouldValidatePartitionCols: Boolean,
+                                   slashSeparatedDatePartitioning: Boolean): 
Array[Object] = {
     if (partitionColumns.length == 0) {
       // This is a non-partitioned table
       Array.empty
@@ -261,6 +266,8 @@ object HoodieSparkUtils extends SparkAdapterSupport with 
SparkVersionsSupport wi
           val partitionValue = if (partitionPath.startsWith(prefix)) {
             // support hive style partition path
             partitionPath.substring(prefix.length)
+          } else if (slashSeparatedDatePartitioning) {
+            partitionPath.replace('/', '-')
           } else {
             partitionPath
           }
diff --git 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
index 17b255738c43..3bb69e0424fb 100644
--- 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
+++ 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
@@ -223,4 +223,44 @@ public class TestComplexKeyGenerator extends 
KeyGeneratorTestUtilities {
       assertEquals(UTF8String.fromString(partitionPath), 
compositeKeyGenerator.getPartitionPath(internalRow, row.schema()));
     }
   }
+
+  @Test
+  void testSlashSeparatedDatePartitioning() {
+    TypedProperties properties = new TypedProperties();
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), 
"timestamp");
+    
properties.put(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(), 
"true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), 
"false");
+
+    ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(properties);
+
+    // Create a record with date in yyyy-MM-dd format
+    GenericRecord avroRecord = KeyGeneratorTestUtilities.getRecord();
+    avroRecord.put("timestamp", "2026-01-05");
+
+    // The partition path should be transformed to yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    assertEquals("_row_key:key1", key.getRecordKey());
+    assertEquals("2026/01/05", key.getPartitionPath());
+  }
+
+  @Test
+  void testSlashSeparatedDatePartitioningWithAlreadyFormattedInput() {
+    TypedProperties properties = new TypedProperties();
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), 
"timestamp");
+    
properties.put(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(), 
"true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), 
"false");
+
+    ComplexKeyGenerator keyGenerator = new ComplexKeyGenerator(properties);
+
+    // Create a record with date already in yyyy/MM/dd format
+    GenericRecord avroRecord = KeyGeneratorTestUtilities.getRecord();
+    avroRecord.put("timestamp", "2026/01/01");
+
+    // The partition path should remain in yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    assertEquals("_row_key:key1", key.getRecordKey());
+    assertEquals("2026/01/01", key.getPartitionPath());
+  }
 }
diff --git 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
index 773ad3218e3b..6e526ccb6c73 100644
--- 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
+++ 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
@@ -414,4 +414,48 @@ class TestCustomKeyGenerator extends 
KeyGeneratorTestUtilities {
     // custom key generator will fail in the constructor, and we must unwrap 
the cause for asserting error messages
     return e.getCause().getCause();
   }
+
+  @Test
+  void testSlashSeparatedDatePartitioning() {
+    TypedProperties properties = new TypedProperties();
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), 
"timestamp:simple");
+    
properties.put(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(), 
"true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), 
"false");
+    properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), 
CustomKeyGenerator.class.getName());
+
+    BuiltinKeyGenerator keyGenerator =
+        (BuiltinKeyGenerator) 
HoodieSparkKeyGeneratorFactory.createKeyGenerator(properties);
+
+    // Create a record with date in yyyy-MM-dd format
+    GenericRecord avroRecord = KeyGeneratorTestUtilities.getRecord();
+    avroRecord.put("timestamp", "2026-01-05");
+
+    // The partition path should be transformed to yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    assertEquals("key1", key.getRecordKey());
+    assertEquals("2026/01/05", key.getPartitionPath());
+  }
+
+  @Test
+  void testSlashSeparatedDatePartitioningWithAlreadyFormattedInput() {
+    TypedProperties properties = new TypedProperties();
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), 
"timestamp:simple");
+    
properties.put(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(), 
"true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), 
"false");
+    properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), 
CustomKeyGenerator.class.getName());
+
+    BuiltinKeyGenerator keyGenerator =
+        (BuiltinKeyGenerator) 
HoodieSparkKeyGeneratorFactory.createKeyGenerator(properties);
+
+    // Create a record with date already in yyyy/MM/dd format
+    GenericRecord avroRecord = KeyGeneratorTestUtilities.getRecord();
+    avroRecord.put("timestamp", "2026/01/01");
+
+    // The partition path should remain in yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    assertEquals("key1", key.getRecordKey());
+    assertEquals("2026/01/01", key.getPartitionPath());
+  }
 }
diff --git 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
index 5c277d2e7b5d..5635f0297a5a 100644
--- 
a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
+++ 
b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
@@ -19,8 +19,10 @@
 
 package org.apache.hudi.keygen;
 
+import org.apache.avro.generic.GenericData;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.schema.HoodieSchema;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieKeyException;
 import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
@@ -90,6 +92,15 @@ class TestSimpleKeyGenerator extends 
KeyGeneratorTestUtilities {
     return properties;
   }
 
+  private TypedProperties getPropsWithSlashSeparatedDatePartitioning() {
+    TypedProperties properties = new TypedProperties();
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), 
"timestamp");
+    
properties.put(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(), 
"true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(), 
"false");
+    return properties;
+  }
+
   @Test
   void testNullPartitionPathFields() {
     assertThrows(IllegalArgumentException.class, () -> new 
SimpleKeyGenerator(getPropertiesWithoutPartitionPathProp()));
@@ -168,4 +179,38 @@ class TestSimpleKeyGenerator extends 
KeyGeneratorTestUtilities {
     Assertions.assertEquals("key1", keyGenerator.getRecordKey(row));
     Assertions.assertEquals(expectedPartitionPath, 
keyGenerator.getPartitionPath(row));
   }
+
+  @Test
+  void testSlashSeparatedDatePartitioning() {
+    SimpleKeyGenerator keyGenerator = new 
SimpleKeyGenerator(getPropsWithSlashSeparatedDatePartitioning());
+
+    // Create a record with date in yyyy-MM-dd format
+    GenericRecord avroRecord = new 
GenericData.Record(HoodieSchema.parse(KeyGeneratorTestUtilities.EXAMPLE_SCHEMA).getAvroSchema());
+    avroRecord.put("timestamp", "2026-01-05");
+    avroRecord.put("_row_key", "key1");
+    avroRecord.put("ts_ms", "2026-01-05");
+    avroRecord.put("pii_col", "val1");
+
+    // The partition path should be transformed to yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    Assertions.assertEquals("key1", key.getRecordKey());
+    Assertions.assertEquals("2026/01/05", key.getPartitionPath());
+  }
+
+  @Test
+  void testSlashSeparatedDatePartitioningWithAlreadyFormattedInput() {
+    SimpleKeyGenerator keyGenerator = new 
SimpleKeyGenerator(getPropsWithSlashSeparatedDatePartitioning());
+
+    // Create a record with date already in yyyy/MM/dd format
+    GenericRecord avroRecord = new 
GenericData.Record(HoodieSchema.parse(KeyGeneratorTestUtilities.EXAMPLE_SCHEMA).getAvroSchema());
+    avroRecord.put("timestamp", "2026/01/01");
+    avroRecord.put("_row_key", "key1");
+    avroRecord.put("ts_ms", "2026/01/01");
+    avroRecord.put("pii_col", "val1");
+
+    // The partition path should remain in yyyy/MM/dd format
+    HoodieKey key = keyGenerator.getKey(avroRecord);
+    Assertions.assertEquals("key1", key.getRecordKey());
+    Assertions.assertEquals("2026/01/01", key.getPartitionPath());
+  }
 }
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
index cfcf203ebc40..3de45e1754ee 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
@@ -367,6 +367,7 @@ public class HoodieTableConfig extends HoodieConfig {
 
   public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = 
KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
   public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE = 
KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
+  public static final ConfigProperty<String> SLASH_SEPARATED_DATE_PARTITIONING 
= KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING;
 
   public static final List<ConfigProperty<String>> PERSISTED_CONFIG_LIST = 
Arrays.asList(
       TIMESTAMP_TYPE_FIELD,
@@ -1211,6 +1212,10 @@ public class HoodieTableConfig extends HoodieConfig {
     return 
HoodieTimelineTimeZone.valueOf(getStringOrDefault(TIMELINE_TIMEZONE));
   }
 
+  public boolean getSlashSeparatedDatePartitioning() {
+    return 
getBooleanOrDefault(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING);
+  }
+
   public String getHiveStylePartitioningEnable() {
     return getStringOrDefault(HIVE_STYLE_PARTITIONING_ENABLE);
   }
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
index 33653c11ffd3..6ddd459e3d6b 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
@@ -1071,6 +1071,7 @@ public class HoodieTableMetaClient implements 
Serializable {
     private Boolean populateMetaFields;
     private String keyGeneratorClassProp;
     private String keyGeneratorType;
+    private Boolean slashSeparatedDatePartitioning;
     private Boolean hiveStylePartitioningEnable;
     private Boolean urlEncodePartitioning;
     private HoodieTimelineTimeZone commitTimeZone;
@@ -1237,6 +1238,11 @@ public class HoodieTableMetaClient implements 
Serializable {
       return this;
     }
 
+    public TableBuilder setSlashSeparatedDatePartitioning(Boolean 
slashSeparatedDatePartitioning) {
+      this.slashSeparatedDatePartitioning = slashSeparatedDatePartitioning;
+      return this;
+    }
+
     public TableBuilder setHiveStylePartitioningEnable(Boolean 
hiveStylePartitioningEnable) {
       this.hiveStylePartitioningEnable = hiveStylePartitioningEnable;
       return this;
@@ -1436,6 +1442,9 @@ public class HoodieTableMetaClient implements 
Serializable {
       } else if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_TYPE)) {
         
setKeyGeneratorClassProp(KeyGeneratorType.valueOf(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_TYPE)).getClassName());
       }
+      if 
(hoodieConfig.contains(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING)) {
+        
setSlashSeparatedDatePartitioning(hoodieConfig.getBoolean(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING));
+      }
       if 
(hoodieConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)) {
         
setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
       }
@@ -1576,6 +1585,9 @@ public class HoodieTableMetaClient implements 
Serializable {
         KeyGeneratorType type = KeyGeneratorType.valueOf(keyGeneratorType);
         tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_TYPE, 
type.name());
       }
+      if (null != slashSeparatedDatePartitioning) {
+        
tableConfig.setValue(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING, 
Boolean.toString(slashSeparatedDatePartitioning));
+      }
       if (null != hiveStylePartitioningEnable) {
         tableConfig.setValue(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, 
Boolean.toString(hiveStylePartitioningEnable));
       }
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java 
b/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
index 3c19ae5cfe9d..074061cd91de 100644
--- a/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
+++ b/hudi-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
@@ -39,6 +39,7 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
   protected List<String> recordKeyFields;
   @Getter
   protected List<String> partitionPathFields;
+  protected final boolean slashSeparatedDatePartitioning;
   protected final boolean encodePartitionPath;
   protected final boolean hiveStylePartitioning;
   @Getter
@@ -46,6 +47,8 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
 
   protected BaseKeyGenerator(TypedProperties config) {
     super(config);
+    this.slashSeparatedDatePartitioning = 
config.getBoolean(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key(),
+        
Boolean.parseBoolean(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.defaultValue()));
     this.encodePartitionPath = 
config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.key(),
         
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING.defaultValue()));
     this.hiveStylePartitioning = 
config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE.key(),
diff --git 
a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
 
b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
index 1a45dd539838..d2c362a971ef 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
@@ -34,6 +34,14 @@ import 
org.apache.hudi.common.config.TimestampKeyGeneratorConfig;
     description = "")
 public class KeyGeneratorOptions extends HoodieConfig {
 
+  public static final ConfigProperty<String> SLASH_SEPARATED_DATE_PARTITIONING 
= ConfigProperty
+      .key("hoodie.datasource.write.slash.separated.date.partitioning")
+      .defaultValue("false")
+      .markAdvanced()
+      .withDocumentation("Flag to indicate whether to use slash separated date 
partitioning.\n"
+          + "If set to true, date partition values in yyyy-MM-dd format will 
be transformed to yyyy/MM/dd directory structure.\n"
+          + "By default false. Cannot be used together with hive-style 
partitioning.");
+
   public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = 
ConfigProperty
       .key("hoodie.datasource.write.partitionpath.urlencode")
       .defaultValue("false")
diff --git 
a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
 
b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
index b240bb2dfabd..8edb3132b728 100644
--- 
a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
+++ 
b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
@@ -384,7 +384,7 @@ class TestHoodieTableConfig extends HoodieCommonTestHarness 
{
   @Test
   void testDefinedTableConfigs() {
     List<ConfigProperty<?>> configProperties = 
HoodieTableConfig.definedTableConfigs();
-    assertEquals(42, configProperties.size());
+    assertEquals(43, configProperties.size());
     configProperties.forEach(c -> {
       assertNotNull(c);
       assertFalse(c.doc().isEmpty());
diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala
index cd79b78ddb81..567053c618a7 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala
@@ -410,11 +410,13 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
     val (staticPartitionColumnNames, staticPartitionColumnValues) = 
staticPartitionColumnNameValuePairs.unzip
 
     val hiveStylePartitioning = 
metaClient.getTableConfig.getHiveStylePartitioningEnable.toBoolean
+    val slashSeparatedDatePartitioning = 
metaClient.getTableConfig.getSlashSeparatedDatePartitioning
 
     val partitionPathFormatter = new StringPartitionPathFormatter(
       JFunction.toJavaSupplier(() => new 
StringPartitionPathFormatter.JavaStringBuilder()),
       hiveStylePartitioning,
-      arePartitionPathsUrlEncoded
+      arePartitionPathsUrlEncoded,
+      slashSeparatedDatePartitioning
     )
 
     partitionPathFormatter.combine(staticPartitionColumnNames.asJava,
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 5f9ed31a0cda..d461de145e5d 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
@@ -23,9 +23,10 @@ import org.apache.hudi.HoodieWriterUtils._
 import org.apache.hudi.common.config.{DFSPropertiesConfiguration, 
TypedProperties}
 import org.apache.hudi.common.model.HoodieTableType
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
-import org.apache.hudi.common.table.HoodieTableConfig.URL_ENCODE_PARTITIONING
+import 
org.apache.hudi.common.table.HoodieTableConfig.{HIVE_STYLE_PARTITIONING_ENABLE, 
URL_ENCODE_PARTITIONING}
 import org.apache.hudi.common.table.timeline.TimelineUtils
 import org.apache.hudi.common.util.StringUtils
+import org.apache.hudi.common.util.ValidationUtils
 import org.apache.hudi.common.util.ValidationUtils.checkArgument
 import org.apache.hudi.config.HoodieWriteConfig
 import org.apache.hudi.hadoop.fs.HadoopFSUtils
@@ -293,6 +294,11 @@ class HoodieCatalogTable(val spark: SparkSession, var 
table: CatalogTable) exten
   private def extraTableConfig(tableExists: Boolean,
                                originTableConfig: Map[String, String] = 
Map.empty,
                                sqlOptions: Map[String, String] = Map.empty): 
Map[String, String] = {
+    ValidationUtils.checkArgument(
+      !(sqlOptions.contains(HIVE_STYLE_PARTITIONING_ENABLE.key)
+        && 
sqlOptions.contains(KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key)),
+      s"Table configs cannot contain both 
${HIVE_STYLE_PARTITIONING_ENABLE.key} "
+        + s"and ${KeyGeneratorOptions.SLASH_SEPARATED_DATE_PARTITIONING.key}")
     val extraConfig = mutable.Map.empty[String, String]
     if (tableExists) {
       val allPartitionPaths = getPartitionPaths
@@ -310,6 +316,14 @@ class HoodieCatalogTable(val spark: SparkSession, var 
table: CatalogTable) exten
         extraConfig(URL_ENCODE_PARTITIONING.key) =
           String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table))
       }
+      if 
(originTableConfig.contains(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key))
 {
+        extraConfig(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key) =
+          
originTableConfig(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key)
+      }
+    } else if 
(sqlOptions.contains(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key)) {
+      extraConfig(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key) =
+        
String.valueOf(sqlOptions(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key))
+      extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = 
"false"
     } else {
       extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = 
"true"
       extraConfig(URL_ENCODE_PARTITIONING.key) = 
URL_ENCODE_PARTITIONING.defaultValue()
diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
index 541239b7991a..29368c8f02c7 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala
@@ -91,6 +91,19 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
       .toMap
   }
 
+  /**
+   * Determine whether slash separated date partitioning is enabled
+   */
+  def isSlashSeparatedDatePartitioning(partitionPaths: Seq[String], table: 
CatalogTable): Boolean = {
+    if (table.partitionColumnNames.nonEmpty) {
+      partitionPaths.forall(partitionPath => {
+        table.partitionColumnNames.size == 1 && 
partitionPath.split("/").length == 3
+      })
+    } else {
+      false
+    }
+  }
+
   /**
    * This method is used to compatible with the old non-hive-styled partition 
table.
    * By default we enable the "hoodie.datasource.write.hive_style_partitioning"
diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
index 9b361ac3d79c..90dfb4f83d72 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
@@ -23,6 +23,7 @@ import org.apache.hudi.common.table.HoodieTableConfig
 import org.apache.hudi.common.util.ConfigUtils
 import org.apache.hudi.exception.{HoodieException, HoodieValidationException}
 import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions
 
 import org.apache.hadoop.fs.Path
 import org.apache.spark.{SPARK_VERSION, SparkConf}
@@ -133,6 +134,7 @@ object CreateHoodieTableCommand {
       checkTableConfigEqual(originTableConfig, tableOptions, 
HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
       checkTableConfigEqual(originTableConfig, tableOptions, 
HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
       checkTableConfigEqual(originTableConfig, tableOptions, 
HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
+      checkTableConfigEqual(originTableConfig, tableOptions, 
HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key)
     }
   }
 
diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
index a7f72780e718..3df7c6673961 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/SqlKeyGenerator.scala
@@ -153,8 +153,7 @@ class SqlKeyGenerator(props: TypedProperties) extends 
BuiltinKeyGenerator(props)
       // in this case.
       if (partitionFragments.size != partitionSchema.get.size) {
         partitionPath
-      }
-      else {
+      } else {
         partitionFragments.zip(partitionSchema.get.fields).map {
           case (partitionValue, partitionField) =>
             val hiveStylePrefix = s"${partitionField.name}="
@@ -162,8 +161,9 @@ class SqlKeyGenerator(props: TypedProperties) extends 
BuiltinKeyGenerator(props)
             val _partitionValue = if (isHiveStyle) 
partitionValue.substring(hiveStylePrefix.length) else partitionValue
             if (_partitionValue == 
PartitionPathEncodeUtils.DEFAULT_PARTITION_PATH) {
               partitionValue
-            }
-            else {
+            } else if (slashSeparatedDatePartitioning) {
+              partitionValue.replace('/', '-')
+            } else {
               partitionField.dataType match {
                 case TimestampType =>
                   val timeMs = if (rowType) { // In RowType, the 
partitionPathValue is the time format string, convert to millis
diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala
 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala
new file mode 100644
index 000000000000..34e6dffd1a13
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hudi.common
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.common.config.HoodieMetadataConfig
+import org.apache.hudi.common.table.HoodieTableMetaClient
+import org.apache.hudi.hadoop.fs.HadoopFSUtils
+import org.apache.hudi.metadata.HoodieBackedTableMetadata
+import org.apache.hudi.storage.{HoodieStorage, StoragePath}
+
+import org.junit.jupiter.api.Assertions.assertTrue
+
+class TestSlashSeparatedPartitionValue extends HoodieSparkSqlTestBase {
+
+  test("Test slash separated date partitions") {
+    withTempDir { tmp =>
+      val targetTable = generateTableName
+      val tablePath = s"${tmp.getCanonicalPath}/$targetTable"
+
+      spark.sql(
+        s"""
+           |create table $targetTable (
+           |  `id` string,
+           |  `name` string,
+           |  `ts` bigint,
+           |  `datestr` STRING
+           |) using hudi
+           | tblproperties (
+           |  'primaryKey' = 'id',
+           |  'type' = 'COW',
+           |  'preCombineField'='ts',
+           |  
'hoodie.datasource.write.slash.separated.date.partitioning'='true'
+           | )
+           | partitioned by (`datestr`)
+           | location '$tablePath'
+        """.stripMargin)
+
+      spark.sql(
+        s"""
+           | insert into $targetTable values
+           | (1, 'a1', 1000, "2026-01-05"),
+           | (2, 'a2', 2000, "2026-01-06")
+        """.stripMargin)
+
+      // check result after insert and merge data into target table
+      checkAnswer(s"select id, name, ts, _hoodie_partition_path, datestr from 
$targetTable limit 10")(
+        Seq("1", "a1", 1000, "2026/01/05", "2026-01-05"),
+        Seq("2", "a2", 2000, "2026/01/06", "2026-01-06")
+      )
+
+      // Verify table config has slash separated date partitioning enabled
+      val metaClient = HoodieTableMetaClient.builder()
+        
.setConf(HadoopFSUtils.getStorageConfWithCopy(spark.sparkContext.hadoopConfiguration))
+        .setBasePath(tablePath)
+        .build()
+      val tableConfig = metaClient.getTableConfig
+      assertTrue(tableConfig.getSlashSeparatedDatePartitioning,
+        "Table config should have slash separated date partitioning enabled")
+
+      // Verify that partition paths are created with slash separated date 
format (yyyy/MM/dd)
+      assertTrue(metaClient.getStorage.exists(new StoragePath(tablePath, 
"2026/01/05")),
+        s"Partition path 2026/01/05 should exist")
+      assertTrue(metaClient.getStorage.exists(new StoragePath(tablePath, 
"2026/01/06")),
+        s"Partition path 2026/01/06 should exist")
+
+      val engine = new HoodieSparkEngineContext(spark.sparkContext)
+      val storage = metaClient.getStorage()
+      val metadataConfig = HoodieMetadataConfig.newBuilder().build()
+      val metadataTable = new HoodieBackedTableMetadata(engine, storage, 
metadataConfig, tablePath)
+      val partitionPaths = metadataTable.getAllPartitionPaths
+      assertTrue(partitionPaths.contains("2026/01/05"))
+      assertTrue(partitionPaths.contains("2026/01/06"))
+      metadataTable.close()
+    }
+  }
+
+  test("Test slash separated date partitions with already formatted input") {
+    Seq(true, false).foreach { slashSeparatedPartitioning =>
+      withTempDir { tmp =>
+        val targetTable = generateTableName
+        val tablePath = s"${tmp.getCanonicalPath}/$targetTable"
+
+        spark.sql(
+          s"""
+             |create table $targetTable (
+             |  `id` string,
+             |  `name` string,
+             |  `ts` bigint,
+             |  `datestr` STRING
+             |) using hudi
+             | tblproperties (
+             |  'primaryKey' = 'id',
+             |  'type' = 'COW',
+             |  'preCombineField'='ts',
+             |  
'hoodie.datasource.write.slash.separated.date.partitioning'='$slashSeparatedPartitioning'
+             | )
+             | partitioned by (`datestr`)
+             | location '$tablePath'
+          """.stripMargin)
+
+        spark.sql(
+          s"""
+             | insert into $targetTable values
+             | (1, 'a1', 1000, "2026/01/01"),
+             | (2, 'a2', 2000, "2026/01/02")
+          """.stripMargin)
+
+        val (firstPartitionValue, secondPartitionValue) = if 
(slashSeparatedPartitioning) {
+          ("2026-01-01", "2026-01-02")
+        } else {
+          ("2026/01/01", "2026/01/02")
+        }
+        // check result after insert - already formatted values should remain 
as is
+        checkAnswer(s"select id, name, ts, _hoodie_partition_path, datestr 
from $targetTable limit 10")(
+          Seq("1", "a1", 1000, "2026/01/01", firstPartitionValue),
+          Seq("2", "a2", 2000, "2026/01/02", secondPartitionValue)
+        )
+
+        // Verify table config
+        val metaClient = HoodieTableMetaClient.builder()
+          
.setConf(HadoopFSUtils.getStorageConfWithCopy(spark.sparkContext.hadoopConfiguration))
+          .setBasePath(tablePath)
+          .build()
+        val tableConfig = metaClient.getTableConfig
+        assertTrue(tableConfig.getSlashSeparatedDatePartitioning == 
slashSeparatedPartitioning,
+          s"Table config should have slash separated date partitioning set to 
$slashSeparatedPartitioning")
+
+        // Verify that partition paths are created with slash separated date 
format (yyyy/MM/dd)
+        assertTrue(metaClient.getStorage.exists(new StoragePath(tablePath, 
"2026/01/01")),
+          s"Partition path 2026/01/01 should exist")
+        assertTrue(metaClient.getStorage.exists(new StoragePath(tablePath, 
"2026/01/02")),
+          s"Partition path 2026/01/02 should exist")
+
+        val engine = new HoodieSparkEngineContext(spark.sparkContext)
+        val storage = metaClient.getStorage()
+        val metadataConfig = HoodieMetadataConfig.newBuilder().build()
+        val metadataTable = new HoodieBackedTableMetadata(engine, storage, 
metadataConfig, tablePath)
+        val partitionPaths = metadataTable.getAllPartitionPaths
+        assertTrue(partitionPaths.contains("2026/01/01"))
+        assertTrue(partitionPaths.contains("2026/01/02"))
+        metadataTable.close()
+      }
+    }
+  }
+}
diff --git 
a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java
 
b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java
index f23c7584e653..890c9712f714 100644
--- 
a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java
+++ 
b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java
@@ -51,6 +51,7 @@ import static 
org.apache.hudi.common.config.HoodieCommonConfig.META_SYNC_BASE_PA
 import static 
org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
 import static org.apache.hudi.common.table.HoodieTableConfig.BASE_FILE_FORMAT;
 import static org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME;
+import static 
org.apache.hudi.common.table.HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING;
 import static 
org.apache.hudi.common.table.HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE;
 import static 
org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_NAME_KEY;
 import static 
org.apache.hudi.common.table.HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY;
@@ -136,6 +137,9 @@ public class HoodieSyncConfig extends HoodieConfig {
             if (cfg.contains(HIVE_STYLE_PARTITIONING_ENABLE)
                 && 
cfg.getString(HIVE_STYLE_PARTITIONING_ENABLE).equals("true")) {
               return 
Option.of("org.apache.hudi.hive.HiveStylePartitionValueExtractor");
+            } else if (cfg.contains(SLASH_SEPARATED_DATE_PARTITIONING)
+                && 
cfg.getString(SLASH_SEPARATED_DATE_PARTITIONING).equals("true")) {
+              return 
Option.of("org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor");
             } else {
               return 
Option.of("org.apache.hudi.hive.SinglePartPartitionValueExtractor");
             }

Reply via email to