nsivabalan commented on code in PR #17787:
URL: https://github.com/apache/hudi/pull/17787#discussion_r2738583570


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -1211,6 +1212,10 @@ public HoodieTimelineTimeZone getTimelineTimezone() {
     return 
HoodieTimelineTimeZone.valueOf(getStringOrDefault(TIMELINE_TIMEZONE));
   }
 
+  public String getSlashSeparatedDatePartitioning() {

Review Comment:
   we should return boolean directly. 



##########
hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java:
##########
@@ -34,6 +34,12 @@
     description = "")
 public class KeyGeneratorOptions extends HoodieConfig {
 
+  public static final ConfigProperty<String> SLASH_SEPARATED_DATE_PARTITIONING 
= ConfigProperty
+      .key("hoodie.datasource.write.slash.separated.date.partitionpath")

Review Comment:
   `hoodie.datasource.write.slash.separated.date.partitioning` 



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -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 && 
isValidDate(partitionPath)) {

Review Comment:
   do we need to validate for every value here? 
   is it possible to add the validate during writes. 
   if writes are good, then we do not need any validation on reads right ?



##########
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 URL encoding is enabled
+   */
+  def isHierarchicalDatePartitioning(partitionPaths: Seq[String], table: 
CatalogTable): Boolean = {

Review Comment:
   again, lets try to maintain same naming across.



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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
+
+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.partitionpath'='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, datestr from $targetTable limit 10")(

Review Comment:
   I don't see the slash seperated values at all in the validation. 
   we should add the meta field as well and assert the values



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/catalyst/catalog/HoodieCatalogTable.scala:
##########
@@ -310,6 +316,17 @@ 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 {
+        extraConfig(HoodieTableConfig.SLASH_SEPARATED_DATE_PARTITIONING.key) =
+          String.valueOf(isHierarchicalDatePartitioning(allPartitionPaths, 
table))

Review Comment:
   is the `else` block really required. 
   I understand why do we have a similar code block for hive style 
partitioning. its mainly to support an already existing table. 
   



##########
hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java:
##########
@@ -34,6 +34,12 @@
     description = "")
 public class KeyGeneratorOptions extends HoodieConfig {
 
+  public static final ConfigProperty<String> SLASH_SEPARATED_DATE_PARTITIONING 
= ConfigProperty
+      .key("hoodie.datasource.write.slash.separated.date.partitionpath")
+      .defaultValue("false")
+      .markAdvanced()
+      .withDocumentation("Should we url encode the partition path value, 
before creating the folder structure.");

Review Comment:
   fix documentation



##########
hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java:
##########
@@ -34,6 +34,12 @@
     description = "")
 public class KeyGeneratorOptions extends HoodieConfig {
 
+  public static final ConfigProperty<String> SLASH_SEPARATED_DATE_PARTITIONING 
= ConfigProperty
+      .key("hoodie.datasource.write.slash.separated.date.partitionpath")
+      .defaultValue("false")
+      .markAdvanced()
+      .withDocumentation("Should we url encode the partition path value, 
before creating the folder structure.");

Review Comment:
   lets add fromVersion annotation as well



##########
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 hierarchicalDatePartitioning = 
metaClient.getTableConfig.getSlashSeparatedDatePartitioning.toBoolean

Review Comment:
   can we maintain same naming. if not causes confusion. 
   `hierarchicalDatePartitioning` -> `slashSeparatedDatePartitioning`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to