diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java index bb6802fa99c8e..5d0bae2bd1bda 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/keygen/TimestampBasedKeyGenerator.java @@ -39,7 +39,7 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator { enum TimestampType implements Serializable { - UNIX_TIMESTAMP, DATE_STRING, MIXED + UNIX_TIMESTAMP, DATE_STRING, MIXED, EPOCHMILLISECONDS } private final TimestampType timestampType; @@ -97,9 +97,10 @@ public HoodieKey getKey(GenericRecord record) { throw new HoodieNotSupportedException( "Unexpected type for partition field: " + partitionVal.getClass().getName()); } + Date timestamp = this.timestampType == TimestampType.EPOCHMILLISECONDS ? new Date(unixTime) : new Date(unixTime * 1000); return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField), - partitionPathFormat.format(new Date(unixTime * 1000))); + partitionPathFormat.format(timestamp)); } catch (ParseException pe) { throw new HoodieDeltaStreamerException("Unable to parse input partition field :" + partitionVal, pe); }