hudi
hudi copied to clipboard
[SUPPORT] Slashes in partition columns
When the partition column contains the slash character ("/"), Hudi could write the data incorrectly or do not read the back.
Test (I use some helpers to write and read Hudi data, they write write data to the local FS and read it):
class HudiPartitionPathTest extends AnyFlatSpec with Matchers with TestHudi {
"Partition paths" should "be generated properly" in {
val data = rows(
(1, "one", "partition with space"),
(2, "two", "partition with space"),
(3, "three", "partition-with-dashes"),
(4, "four", "partition-with-dashes"),
(5, "five", "partition="),
(6, "six", "partition="),
(7, "seven", "partition%"),
(8, "eight", "partition%"),
(9, "nine", "partition"),
(10, "ten", "partition"),
(11, "eleven", "partition/"),
(12, "twelve", "partition/"),
(13, "thirteen", "partition/slaanesh"),
(14, "fourteen", "partition/slaanesh")
)
val path = createHudiDataset(
getClass.getName,
data,
schema("id" -> "int", "value" -> "string", "partition" -> "string")
)
val fsPartitionPaths = allFSPartitionPaths(new File(path), path.length + 1)
println(fsPartitionPaths mkString "\n")
val df = readHudiDataset(path)
df show false
val partitionPaths = df.select("_hoodie_partition_path").dropDuplicates.collect().map(_.getString(0)).toSet
fsPartitionPaths shouldEqual partitionPaths
}
private val filter: FilenameFilter = (_, name) => !name.startsWith(".")
def allFSPartitionPaths(dir: File, prefixLength: Int): Set[String] =
(dir.listFiles(filter) foldLeft Set.empty[String]) { (paths, file) =>
if (file.isFile) paths + file.getParent.substring(prefixLength)
else paths | allFSPartitionPaths(file, prefixLength)
}
}
The output is
daas_date=partition
daas_date=partition-with-dashes
daas_date=partition with space
daas_date=partition%
daas_date=partition/slaanesh
daas_date=partition=
+-------------------+---------------------+------------------+-------------------------------+-----------------------------------------------------------------------+---+------+---------------------+-------------------+
|_hoodie_commit_time|_hoodie_commit_seqno |_hoodie_record_key|_hoodie_partition_path |_hoodie_file_name |id |value |daas_date |daas_internal_ts |
+-------------------+---------------------+------------------+-------------------------------+-----------------------------------------------------------------------+---+------+---------------------+-------------------+
|20240226105035783 |20240226105035783_6_0|11 |daas_date=partition/ |961e9ba8-acef-4957-9357-7032702f1bb2-0_6-4-16_20240226105035783.parquet|11 |eleven|partition/ |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_6_1|12 |daas_date=partition/ |961e9ba8-acef-4957-9357-7032702f1bb2-0_6-4-16_20240226105035783.parquet|12 |twelve|partition/ |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_1_0|9 |daas_date=partition |9f1a3606-25a2-4e79-93fe-f0c03c86cf3d-0_1-4-11_20240226105035783.parquet|9 |nine |partition |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_1_1|10 |daas_date=partition |9f1a3606-25a2-4e79-93fe-f0c03c86cf3d-0_1-4-11_20240226105035783.parquet|10 |ten |partition |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_2_0|3 |daas_date=partition-with-dashes|49d7ac36-67ba-44df-be90-9d73b51fdbc2-0_2-4-12_20240226105035783.parquet|3 |three |partition-with-dashes|2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_2_1|4 |daas_date=partition-with-dashes|49d7ac36-67ba-44df-be90-9d73b51fdbc2-0_2-4-12_20240226105035783.parquet|4 |four |partition-with-dashes|2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_3_0|1 |daas_date=partition with space |9d33b484-0e8c-4299-ab8b-235d6c3a3d5d-0_3-4-13_20240226105035783.parquet|1 |one |partition with space |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_3_1|2 |daas_date=partition with space |9d33b484-0e8c-4299-ab8b-235d6c3a3d5d-0_3-4-13_20240226105035783.parquet|2 |two |partition with space |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_0_0|7 |daas_date=partition% |8e4bfc08-37f8-4f4e-877b-ff5b2218dbd5-0_0-4-10_20240226105035783.parquet|7 |seven |partition% |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_0_1|8 |daas_date=partition% |8e4bfc08-37f8-4f4e-877b-ff5b2218dbd5-0_0-4-10_20240226105035783.parquet|8 |eight |partition% |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_5_0|5 |daas_date=partition= |1134b457-b713-48e2-ace4-a99981b3efd8-0_5-4-15_20240226105035783.parquet|5 |five |partition= |2024-02-26 08:25:05|
|20240226105035783 |20240226105035783_5_1|6 |daas_date=partition= |1134b457-b713-48e2-ace4-a99981b3efd8-0_5-4-15_20240226105035783.parquet|6 |six |partition= |2024-02-26 08:25:05|
+-------------------+---------------------+------------------+-------------------------------+-----------------------------------------------------------------------+---+------+---------------------+-------------------+
As you can see rows 13 and 14 was not read, and "partition" and "partition/" on the file system have the same path (I am not sure about the impact, but probably there could be performance issues).
Maybe it would be great to quote some characters in partition paths?
Environment Description
-
Hudi version : 0.13.1
-
Storage (HDFS/S3/GCS..): Local FS
you are right, we should enode the partition path for these special characters.
@eshu I tried to insert these values and at least read/write worked fine. I do understand in case of slash it created the inner sub folder. Were you able to make it work by encoding them. Let us know in case you need any other help here or Feel free to close if all good.
columns = ["ts","uuid","rider","driver","fare","city"]
data =[(1695159649087,"334e26e9-8355-45cc-97c6-c31daf0df330","rider-A","driver-K",19.10,"san francisco"),
(1695091554788,"e96c4396-3fad-413a-a942-4cb36106d721","rider-B","driver-L",27.70 ,"san-francisco"),
(1695046462179,"9909a8b1-2d15-4d3d-8ec9-efc48c536a00","rider-C","driver-M",33.90 ,"san_francisco%"),
(1695516137016,"e3cf430c-889d-4015-bc98-59bdce1e530c","rider-C","driver-N",34.15,"sao/paulo")]
spark = get_spark_session(spark_version="3.2", hudi_version="0.13.0")
inserts = spark.createDataFrame(data).toDF(*columns)
hudi_options = {
'hoodie.table.name': tableName,
'hoodie.datasource.write.recordkey.field' : 'uuid',
'hoodie.datasource.write.precombine.field' : 'ts',
'hoodie.datasource.write.partitionpath.field': 'city',
}
# Insert data
inserts.write.format("hudi"). \
options(**hudi_options). \
mode("overwrite"). \
save(basePath)
spark.read.format("hudi").load(basePath).show()
Similar jira raised to fix this issue - https://issues.apache.org/jira/browse/HUDI-7484
@ad1happy2go It does not work in my example. Did you tried it?
Yes I tried this - https://github.com/apache/hudi/issues/10754#issuecomment-1979027421
Can you try the same?
@eshu Any updates on the same?