nsivabalan commented on code in PR #17787:
URL: https://github.com/apache/hudi/pull/17787#discussion_r2739392401
##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java:
##########
@@ -168,4 +179,21 @@ void testNestedPartitionPathField(GenericRecord
nestedColRecord) {
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
Review Comment:
can we add a test where input value is 2026/01/01 -> already rightly
formatted.
Also, I assume, if its random string(say "abc"), we might throw error in
some other layer, but not here where we directly test keyGen class.
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -316,6 +323,16 @@ object HoodieSparkUtils extends SparkAdapterSupport with
SparkVersionsSupport wi
}
}
+ private def isValidDate(str: String): Boolean = {
Review Comment:
is this used anywhere?
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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, datestr from $targetTable limit 10")(
+ Seq("1", "a1", 1000, "2026-01-05"),
+ Seq("2", "a2", 2000, "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"))
Review Comment:
can we close the metadataTable in the end.
##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/TestSlashSeparatedPartitionValue.scala:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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, datestr from $targetTable limit 10")(
Review Comment:
can we also include the partition path meta field and validate
--
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]