@@ -22,7 +22,7 @@ import org.apache.hudi.DataSourceReadOptions._
2222import org .apache .hudi .HoodieConversionUtils .toJavaOption
2323import org .apache .hudi .SparkHoodieTableFileIndex .{deduceQueryType , extractEqualityPredicatesLiteralValues , generateFieldMap , haveProperPartitionValues , shouldListLazily , shouldUsePartitionPathPrefixAnalysis , shouldValidatePartitionColumns }
2424import org .apache .hudi .client .common .HoodieSparkEngineContext
25- import org .apache .hudi .common .config .TypedProperties
25+ import org .apache .hudi .common .config .{ TimestampKeyGeneratorConfig , TypedProperties }
2626import org .apache .hudi .common .model .HoodieRecord .HOODIE_META_COLUMNS_WITH_OPERATION
2727import org .apache .hudi .common .model .{FileSlice , HoodieTableQueryType }
2828import org .apache .hudi .common .table .{HoodieTableMetaClient , TableSchemaResolver }
@@ -46,10 +46,10 @@ import org.apache.spark.sql.catalyst.{InternalRow, expressions}
4646import org .apache .spark .sql .execution .datasources .{FileStatusCache , NoopCache }
4747import org .apache .spark .sql .internal .SQLConf
4848import org .apache .spark .sql .types .{ByteType , DateType , IntegerType , LongType , ShortType , StringType , StructField , StructType }
49+ import org .apache .spark .unsafe .types .UTF8String
4950
5051import java .util .Collections
5152import javax .annotation .concurrent .NotThreadSafe
52-
5353import scala .collection .JavaConverters ._
5454import scala .language .implicitConversions
5555import scala .util .{Success , Try }
@@ -405,9 +405,21 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
405405 }
406406
407407 protected def doParsePartitionColumnValues (partitionColumns : Array [String ], partitionPath : String ): Array [Object ] = {
408- HoodieSparkUtils .parsePartitionColumnValues(partitionColumns, partitionPath, getBasePath, schema,
409- configProperties.getString(DateTimeUtils .TIMEZONE_OPTION , SQLConf .get.sessionLocalTimeZone),
410- sparkParsePartitionUtil, shouldValidatePartitionColumns(spark))
408+ val tableConfig = metaClient.getTableConfig
409+ if (null != tableConfig.getKeyGeneratorClassName
410+ && tableConfig.getKeyGeneratorClassName.equals(classOf [org.apache.hudi.keygen.TimestampBasedKeyGenerator ].getName)
411+ && tableConfig.propsMap.get(TimestampKeyGeneratorConfig .TIMESTAMP_TYPE_FIELD .key())
412+ .matches(" SCALAR|UNIX_TIMESTAMP|EPOCHMILLISECONDS|EPOCHMICROSECONDS" )) {
413+ // For TIMESTAMP key generator when TYPE is SCALAR, UNIX_TIMESTAMP,
414+ // EPOCHMILLISECONDS, or EPOCHMICROSECONDS,
415+ // we couldn't reconstruct initial partition column values from partition paths due to lost data after formatting in most cases.
416+ // But the output for these cases is in a string format, so we can pass partitionPath as UTF8String
417+ Array .fill(partitionColumns.length)(UTF8String .fromString(partitionPath))
418+ } else {
419+ HoodieSparkUtils .parsePartitionColumnValues(partitionColumns, partitionPath, getBasePath, schema,
420+ configProperties.getString(DateTimeUtils .TIMEZONE_OPTION , SQLConf .get.sessionLocalTimeZone),
421+ sparkParsePartitionUtil, shouldValidatePartitionColumns(spark))
422+ }
411423 }
412424
413425 private def arePartitionPathsUrlEncoded : Boolean =
0 commit comments