tub opened a new issue, #5706: URL: https://github.com/apache/paimon/issues/5706
### Search before asking - [x] I searched in the [issues](https://github.com/apache/paimon/issues) and found nothing similar. ### Paimon version Using the FlinkCDC writer, if it encounters a negative unix timestamp - i.e a timestamp representing a date before 1970 - it will throw the following exception: ``` Caused by: java.time.format.DateTimeParseException: Text '-22383318' could not be parsed at index 9 at java.time.format.DateTimeFormatter.parseResolved0(DateTimeFormatter.java:2046) ~[?:?] at java.time.format.DateTimeFormatter.parse(DateTimeFormatter.java:1874) ~[?:?] at org.apache.paimon.utils.DateTimeUtils.parseTimestampData(DateTimeUtils.java:537) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.utils.BinaryStringUtils.toTimestamp(BinaryStringUtils.java:313) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.utils.TypeUtils.castFromStringInternal(TypeUtils.java:163) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.utils.TypeUtils.castFromCdcValueString(TypeUtils.java:97) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.flink.sink.cdc.CdcRecordUtils.projectAsInsert(CdcRecordUtils.java:60) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.flink.sink.cdc.CdcRecordKeyAndBucketExtractor.bucket(CdcRecordKeyAndBucketExtractor.java:97) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.flink.sink.cdc.CdcFixedBucketChannelComputerBase.channel(CdcFixedBucketChannelComputerBase.java:52) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.flink.sink.FlinkStreamPartitioner.selectChannel(FlinkStreamPartitioner.java:48) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.paimon.flink.sink.FlinkStreamPartitioner.selectChannel(FlinkStreamPartitioner.java:32) ~[paimon-flink-1.20-1.1-SNAPSHOT.jar:1.1-SNAPSHOT] at org.apache.flink.runtime.io.network.api.writer.ChannelSelectorRecordWriter.emit(ChannelSelectorRecordWriter.java:55) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.runtime.io.RecordWriterOutput.pushToRecordWriter(RecordWriterOutput.java:140) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.runtime.io.RecordWriterOutput.collectAndCheckIfChained(RecordWriterOutput.java:120) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.runtime.io.RecordWriterOutput.collectAndCheckIfChained(RecordWriterOutput.java:53) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.runtime.tasks.CopyingBroadcastingOutputCollector.collect(CopyingBroadcastingOutputCollector.java:47) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.runtime.tasks.CopyingBroadcastingOutputCollector.collect(CopyingBroadcastingOutputCollector.java:28) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:60) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.api.operators.CountingOutput.collect(CountingOutput.java:32) ~[flink-dist-1.20.1.jar:1.20.1] at org.apache.flink.streaming.api.operators.TimestampedCollector.collect(TimestampedCollector.java:52) ~[flink-dist-1.20.1.jar:1.20.1] ``` This is due to StringUtils.isNumeric returning false as soon as it sees the minus sign. Further changes to BinaryStringUtils.fromMillisToTimestamp are required to handle negative nanoseconds. Relates to #4239 as this exception was also seen there. ### Compute Engine Flink 1.20 ### Minimal reproduce step Call `BinaryStringUtils.toTimestamp(BinaryString.fromString("-123456789", 9))` in a unit test. ### What doesn't meet your expectations? Negative unix timestamps should be parsed correctly rather than an exception thrown. ### Anything else? I have a PR incoming :) ### Are you willing to submit a PR? - [x] I'm willing to submit a PR! -- 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: issues-unsubscr...@paimon.apache.org.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org