danny0405 commented on code in PR #13943:
URL: https://github.com/apache/hudi/pull/13943#discussion_r2366726442
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieSparkUtils.scala:
##########
@@ -332,6 +342,71 @@ object HoodieSparkUtils extends SparkAdapterSupport with
SparkVersionsSupport wi
).toSeq(partitionSchema)
}
+ private def buildPartitionSchemaForNestedFields(schema: StructType,
partitionColumns: Array[String]): StructType = {
+ val partitionFields = partitionColumns.flatMap { partitionCol =>
+ extractNestedField(schema, partitionCol)
+ }
+ StructType(partitionFields)
+ }
+
+ private def extractNestedField(schema: StructType, fieldPath: String):
Option[StructField] = {
+ val pathParts = fieldPath.split("\\.")
+
+ def traverseSchema(currentSchema: StructType, remainingPath: List[String],
originalFieldName: String): Option[StructField] = {
+ remainingPath match {
+ case Nil => None
+ case head :: Nil =>
+ currentSchema.fields.find(_.name == head).map { field =>
+ StructField(originalFieldName, field.dataType, field.nullable,
field.metadata)
+ }
+ case head :: tail =>
+ currentSchema.fields.find(_.name == head) match {
+ case Some(StructField(_, structType: StructType, _, _)) =>
+ traverseSchema(structType, tail, originalFieldName)
+ case _ => None
+ }
+ }
+ }
+ traverseSchema(schema, pathParts.toList, fieldPath)
+ }
+
+ private def castStringToType(value: String, dataType:
org.apache.spark.sql.types.DataType): Any = {
+ import org.apache.spark.sql.types._
+
+ // handling cases where the value contains path separators or is complex
+ if (value.contains("/") || value.contains("=")) {
+ // For complex paths, falling back to string representation
+ logWarning(s"Cannot convert complex partition path '$value' to
$dataType, keeping as string")
+ return UTF8String.fromString(value)
+ }
+
+ try {
+ dataType match {
+ case LongType => value.toLong
+ case IntegerType => value.toInt
+ case ShortType => value.toShort
+ case ByteType => value.toByte
+ case FloatType => value.toFloat
+ case DoubleType => value.toDouble
+ case BooleanType => value.toBoolean
+ case _: DecimalType => new java.math.BigDecimal(value)
Review Comment:
Spark has its own decimal data representation, do we have test case to cover
these data types arleady: decimal, timestamp and date.
--
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]