This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git
The following commit(s) were added to refs/heads/master by this push:
new 44b14eea2e [iceberg] support millisecond timestamps in iceberg
compatibility mode (#6352)
44b14eea2e is described below
commit 44b14eea2e7c9834ea9fedc883c24c4da1a4c7bd
Author: Max Falk <[email protected]>
AuthorDate: Sun Dec 28 05:07:33 2025 +0100
[iceberg] support millisecond timestamps in iceberg compatibility mode
(#6352)
---
docs/content/iceberg/overview.md | 8 +-
.../iceberg/manifest/IcebergConversions.java | 11 +-
.../paimon/iceberg/metadata/IcebergDataField.java | 8 +-
.../manifest/IcebergConversionsTimestampTest.java | 128 +++++++++++++++++++++
.../iceberg/metadata/IcebergDataFieldTest.java | 12 +-
5 files changed, 149 insertions(+), 18 deletions(-)
diff --git a/docs/content/iceberg/overview.md b/docs/content/iceberg/overview.md
index 952bab4008..35dc351905 100644
--- a/docs/content/iceberg/overview.md
+++ b/docs/content/iceberg/overview.md
@@ -96,8 +96,8 @@ Paimon Iceberg compatibility currently supports the following
data types.
| `BINARY` | `binary` |
| `VARBINARY` | `binary` |
| `DATE` | `date` |
-| `TIMESTAMP` (precision 4-6) | `timestamp` |
-| `TIMESTAMP_LTZ` (precision 4-6) | `timestamptz` |
+| `TIMESTAMP` (precision 3-6) | `timestamp` |
+| `TIMESTAMP_LTZ` (precision 3-6) | `timestamptz` |
| `TIMESTAMP` (precision 7-9) | `timestamp_ns` |
| `TIMESTAMP_LTZ` (precision 7-9) | `timestamptz_ns` |
| `ARRAY` | `list` |
@@ -106,6 +106,6 @@ Paimon Iceberg compatibility currently supports the
following data types.
{{< hint info >}}
**Note on Timestamp Types:**
-- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 4 to 6 are mapped
to standard Iceberg timestamp types
+- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 3 to 6 are mapped
to standard Iceberg timestamp types
- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 7 to 9 use
nanosecond precision and require Iceberg v3 format
-{{< /hint >}}
\ No newline at end of file
+{{< /hint >}}
diff --git
a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergConversions.java
b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergConversions.java
index a214ff1c82..4998d0db14 100644
---
a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergConversions.java
+++
b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergConversions.java
@@ -116,8 +116,8 @@ public class IcebergConversions {
private static ByteBuffer timestampToByteBuffer(Timestamp timestamp, int
precision) {
Preconditions.checkArgument(
- precision > 3 && precision <= 6,
- "Paimon Iceberg compatibility only support timestamp type with
precision from 4 to 6.");
+ precision >= 3 && precision <= 6,
+ "Paimon Iceberg compatibility only support timestamp type with
precision from 3 to 6.");
return ByteBuffer.allocate(8)
.order(ByteOrder.LITTLE_ENDIAN)
.putLong(0, timestamp.toMicros());
@@ -157,8 +157,11 @@ public class IcebergConversions {
long timestampLong =
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).getLong();
Preconditions.checkArgument(
- timestampPrecision > 3 && timestampPrecision <= 6,
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 6.");
+ timestampPrecision >= 3 && timestampPrecision <= 6,
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 6.");
+ if (timestampPrecision == 3) {
+ return Timestamp.fromEpochMillis(timestampLong);
+ }
return Timestamp.fromMicros(timestampLong);
default:
throw new UnsupportedOperationException("Cannot deserialize
type: " + type);
diff --git
a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java
b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java
index 3714d59bfb..0facdd47e4 100644
---
a/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java
+++
b/paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java
@@ -174,14 +174,14 @@ public class IcebergDataField {
case TIMESTAMP_WITHOUT_TIME_ZONE:
int timestampPrecision = ((TimestampType)
dataType).getPrecision();
Preconditions.checkArgument(
- timestampPrecision > 3 && timestampPrecision <= 9,
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 9.");
+ timestampPrecision >= 3 && timestampPrecision <= 9,
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 9.");
return timestampPrecision >= 7 ? "timestamp_ns" : "timestamp";
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
int timestampLtzPrecision = ((LocalZonedTimestampType)
dataType).getPrecision();
Preconditions.checkArgument(
- timestampLtzPrecision > 3 && timestampLtzPrecision <=
9,
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 9.");
+ timestampLtzPrecision >= 3 && timestampLtzPrecision <=
9,
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 9.");
return timestampLtzPrecision >= 7 ? "timestamptz_ns" :
"timestamptz";
case ARRAY:
ArrayType arrayType = (ArrayType) dataType;
diff --git
a/paimon-core/src/test/java/org/apache/paimon/iceberg/manifest/IcebergConversionsTimestampTest.java
b/paimon-core/src/test/java/org/apache/paimon/iceberg/manifest/IcebergConversionsTimestampTest.java
new file mode 100644
index 0000000000..677b41d54d
--- /dev/null
+++
b/paimon-core/src/test/java/org/apache/paimon/iceberg/manifest/IcebergConversionsTimestampTest.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.iceberg.manifest;
+
+import org.apache.paimon.data.Timestamp;
+import org.apache.paimon.types.DataTypes;
+
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.stream.Stream;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+class IcebergConversionsTimestampTest {
+
+ @ParameterizedTest
+ @MethodSource("provideTimestampConversionCases")
+ void testTimestampToByteBuffer(int precision, Timestamp input, long
expectedMicros) {
+ ByteBuffer buffer =
IcebergConversions.toByteBuffer(DataTypes.TIMESTAMP(precision), input);
+ assertThat(buffer.order()).isEqualTo(ByteOrder.LITTLE_ENDIAN);
+ assertThat(buffer.getLong(0)).isEqualTo(expectedMicros);
+ }
+
+ private static Stream<Arguments> provideTimestampConversionCases() {
+ Timestamp tsMillis =
+ Timestamp.fromEpochMillis(1682164983524L); //
2023-04-22T13:03:03.524 (p=3)
+ Timestamp tsMicros = Timestamp.fromMicros(1683849603123456L); //
2023-05-12T00:00:03.123456
+
+ return Stream.of(
+ // For p=3..6 we encode microseconds per Iceberg spec
+ Arguments.of(3, tsMillis, 1682164983524000L), // micros from
millis
+ Arguments.of(4, tsMillis, 1682164983524000L),
+ Arguments.of(5, tsMillis, 1682164983524000L),
+ Arguments.of(6, tsMillis, 1682164983524000L),
+ Arguments.of(6, tsMicros, 1683849603123456L)); // passthrough
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideInvalidPrecisions")
+ @DisplayName("Invalid timestamp precisions for ByteBuffer conversion")
+ void testTimestampToByteBufferInvalidPrecisions(int precision) {
+ Timestamp timestamp = Timestamp.fromEpochMillis(1682164983524L);
+
+ assertThatThrownBy(
+ () ->
+ IcebergConversions.toByteBuffer(
+ DataTypes.TIMESTAMP(precision),
timestamp))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining(
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 6.");
+ }
+
+ private static Stream<Arguments> provideInvalidPrecisions() {
+ return Stream.of(
+ Arguments.of(0),
+ Arguments.of(1),
+ Arguments.of(2),
+ Arguments.of(7),
+ Arguments.of(8),
+ Arguments.of(9));
+ }
+
+ // ------------------------------------------------------------------------
+ // toPaimonObject tests
+ // ------------------------------------------------------------------------
+
+ @ParameterizedTest
+ @MethodSource("provideTimestampToPaimonCases")
+ void testToPaimonObjectForTimestamp(int precision, long serializedMicros,
String expectedTs) {
+ byte[] bytes = new byte[8];
+
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).putLong(serializedMicros);
+
+ Timestamp actual =
+ (Timestamp)
+
IcebergConversions.toPaimonObject(DataTypes.TIMESTAMP(precision), bytes);
+
+ assertThat(actual.toString()).isEqualTo(expectedTs);
+ }
+
+ private static Stream<Arguments> provideTimestampToPaimonCases() {
+ return Stream.of(
+ // Provide binary in micros; p=3..6 should all parse as micros
+ Arguments.of(3, -1356022717123L, "1927-01-12T07:01:22.877"),
+ Arguments.of(3, 1713790983524L, "2024-04-22T13:03:03.524"),
+ Arguments.of(6, 1640690931207203L,
"2021-12-28T11:28:51.207203"));
+ }
+
+ @ParameterizedTest
+ @MethodSource("provideInvalidTimestampCases")
+ void testToPaimonObjectTimestampInvalid(int precision, long
serializedMicros) {
+ byte[] bytes = new byte[8];
+
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).putLong(serializedMicros);
+
+ assertThatThrownBy(
+ () ->
+ IcebergConversions.toPaimonObject(
+ DataTypes.TIMESTAMP(precision), bytes))
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessageContaining(
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 6.");
+ }
+
+ private static Stream<Arguments> provideInvalidTimestampCases() {
+ return Stream.of(Arguments.of(0, 1698686153L), Arguments.of(9,
1698686153123456789L));
+ }
+}
diff --git
a/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java
b/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java
index f96a063459..f4aff2a79b 100644
---
a/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java
+++
b/paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java
@@ -211,27 +211,27 @@ class IcebergDataFieldTest {
void testTimestampPrecisionValidation() {
// Test invalid precision (<= 3)
DataField invalidTimestampField =
- new DataField(1, "timestamp", new TimestampType(false, 3));
+ new DataField(1, "timestamp", new TimestampType(false, 2));
assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField))
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining(
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 9");
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 9");
// Test invalid precision (<= 3)
DataField invalidTimestampField2 =
- new DataField(2, "timestamp", new TimestampType(false, 3));
+ new DataField(2, "timestamp", new TimestampType(false, 2));
assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField2))
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining(
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 9");
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 9");
// Test invalid local timezone timestamp precision (<= 3)
DataField invalidTimestampLtzField =
- new DataField(3, "timestamptz", new
LocalZonedTimestampType(false, 3));
+ new DataField(3, "timestamptz", new
LocalZonedTimestampType(false, 2));
assertThatThrownBy(() -> new
IcebergDataField(invalidTimestampLtzField))
.isInstanceOf(IllegalArgumentException.class)
.hasMessageContaining(
- "Paimon Iceberg compatibility only support timestamp
type with precision from 4 to 9");
+ "Paimon Iceberg compatibility only support timestamp
type with precision from 3 to 9");
// Test valid precision boundaries
DataField validTimestamp4 = new DataField(4, "timestamp", new
TimestampType(false, 4));