Skip to content

Commit 44b14ee

Browse files
authored
[iceberg] support millisecond timestamps in iceberg compatibility mode (#6352)
1 parent 09ad28d commit 44b14ee

File tree

5 files changed

+149
-18
lines changed

5 files changed

+149
-18
lines changed

docs/content/iceberg/overview.md

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -96,8 +96,8 @@ Paimon Iceberg compatibility currently supports the following data types.
9696
| `BINARY` | `binary` |
9797
| `VARBINARY` | `binary` |
9898
| `DATE` | `date` |
99-
| `TIMESTAMP` (precision 4-6) | `timestamp` |
100-
| `TIMESTAMP_LTZ` (precision 4-6) | `timestamptz` |
99+
| `TIMESTAMP` (precision 3-6) | `timestamp` |
100+
| `TIMESTAMP_LTZ` (precision 3-6) | `timestamptz` |
101101
| `TIMESTAMP` (precision 7-9) | `timestamp_ns` |
102102
| `TIMESTAMP_LTZ` (precision 7-9) | `timestamptz_ns` |
103103
| `ARRAY` | `list` |
@@ -106,6 +106,6 @@ Paimon Iceberg compatibility currently supports the following data types.
106106

107107
{{< hint info >}}
108108
**Note on Timestamp Types:**
109-
- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 4 to 6 are mapped to standard Iceberg timestamp types
109+
- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 3 to 6 are mapped to standard Iceberg timestamp types
110110
- `TIMESTAMP` and `TIMESTAMP_LTZ` types with precision from 7 to 9 use nanosecond precision and require Iceberg v3 format
111-
{{< /hint >}}
111+
{{< /hint >}}

paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergConversions.java

Lines changed: 7 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -116,8 +116,8 @@ public static ByteBuffer toByteBuffer(DataType type, Object value) {
116116

117117
private static ByteBuffer timestampToByteBuffer(Timestamp timestamp, int precision) {
118118
Preconditions.checkArgument(
119-
precision > 3 && precision <= 6,
120-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6.");
119+
precision >= 3 && precision <= 6,
120+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 6.");
121121
return ByteBuffer.allocate(8)
122122
.order(ByteOrder.LITTLE_ENDIAN)
123123
.putLong(0, timestamp.toMicros());
@@ -157,8 +157,11 @@ public static Object toPaimonObject(DataType type, byte[] bytes) {
157157
long timestampLong =
158158
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).getLong();
159159
Preconditions.checkArgument(
160-
timestampPrecision > 3 && timestampPrecision <= 6,
161-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 6.");
160+
timestampPrecision >= 3 && timestampPrecision <= 6,
161+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 6.");
162+
if (timestampPrecision == 3) {
163+
return Timestamp.fromEpochMillis(timestampLong);
164+
}
162165
return Timestamp.fromMicros(timestampLong);
163166
default:
164167
throw new UnsupportedOperationException("Cannot deserialize type: " + type);

paimon-core/src/main/java/org/apache/paimon/iceberg/metadata/IcebergDataField.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -174,14 +174,14 @@ private static Object toTypeObject(DataType dataType, int fieldId, int depth) {
174174
case TIMESTAMP_WITHOUT_TIME_ZONE:
175175
int timestampPrecision = ((TimestampType) dataType).getPrecision();
176176
Preconditions.checkArgument(
177-
timestampPrecision > 3 && timestampPrecision <= 9,
178-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9.");
177+
timestampPrecision >= 3 && timestampPrecision <= 9,
178+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 9.");
179179
return timestampPrecision >= 7 ? "timestamp_ns" : "timestamp";
180180
case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
181181
int timestampLtzPrecision = ((LocalZonedTimestampType) dataType).getPrecision();
182182
Preconditions.checkArgument(
183-
timestampLtzPrecision > 3 && timestampLtzPrecision <= 9,
184-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9.");
183+
timestampLtzPrecision >= 3 && timestampLtzPrecision <= 9,
184+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 9.");
185185
return timestampLtzPrecision >= 7 ? "timestamptz_ns" : "timestamptz";
186186
case ARRAY:
187187
ArrayType arrayType = (ArrayType) dataType;
Lines changed: 128 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,128 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.iceberg.manifest;
20+
21+
import org.apache.paimon.data.Timestamp;
22+
import org.apache.paimon.types.DataTypes;
23+
24+
import org.junit.jupiter.api.DisplayName;
25+
import org.junit.jupiter.params.ParameterizedTest;
26+
import org.junit.jupiter.params.provider.Arguments;
27+
import org.junit.jupiter.params.provider.MethodSource;
28+
29+
import java.nio.ByteBuffer;
30+
import java.nio.ByteOrder;
31+
import java.util.stream.Stream;
32+
33+
import static org.assertj.core.api.Assertions.assertThat;
34+
import static org.assertj.core.api.Assertions.assertThatThrownBy;
35+
36+
class IcebergConversionsTimestampTest {
37+
38+
@ParameterizedTest
39+
@MethodSource("provideTimestampConversionCases")
40+
void testTimestampToByteBuffer(int precision, Timestamp input, long expectedMicros) {
41+
ByteBuffer buffer = IcebergConversions.toByteBuffer(DataTypes.TIMESTAMP(precision), input);
42+
assertThat(buffer.order()).isEqualTo(ByteOrder.LITTLE_ENDIAN);
43+
assertThat(buffer.getLong(0)).isEqualTo(expectedMicros);
44+
}
45+
46+
private static Stream<Arguments> provideTimestampConversionCases() {
47+
Timestamp tsMillis =
48+
Timestamp.fromEpochMillis(1682164983524L); // 2023-04-22T13:03:03.524 (p=3)
49+
Timestamp tsMicros = Timestamp.fromMicros(1683849603123456L); // 2023-05-12T00:00:03.123456
50+
51+
return Stream.of(
52+
// For p=3..6 we encode microseconds per Iceberg spec
53+
Arguments.of(3, tsMillis, 1682164983524000L), // micros from millis
54+
Arguments.of(4, tsMillis, 1682164983524000L),
55+
Arguments.of(5, tsMillis, 1682164983524000L),
56+
Arguments.of(6, tsMillis, 1682164983524000L),
57+
Arguments.of(6, tsMicros, 1683849603123456L)); // passthrough
58+
}
59+
60+
@ParameterizedTest
61+
@MethodSource("provideInvalidPrecisions")
62+
@DisplayName("Invalid timestamp precisions for ByteBuffer conversion")
63+
void testTimestampToByteBufferInvalidPrecisions(int precision) {
64+
Timestamp timestamp = Timestamp.fromEpochMillis(1682164983524L);
65+
66+
assertThatThrownBy(
67+
() ->
68+
IcebergConversions.toByteBuffer(
69+
DataTypes.TIMESTAMP(precision), timestamp))
70+
.isInstanceOf(IllegalArgumentException.class)
71+
.hasMessageContaining(
72+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 6.");
73+
}
74+
75+
private static Stream<Arguments> provideInvalidPrecisions() {
76+
return Stream.of(
77+
Arguments.of(0),
78+
Arguments.of(1),
79+
Arguments.of(2),
80+
Arguments.of(7),
81+
Arguments.of(8),
82+
Arguments.of(9));
83+
}
84+
85+
// ------------------------------------------------------------------------
86+
// toPaimonObject tests
87+
// ------------------------------------------------------------------------
88+
89+
@ParameterizedTest
90+
@MethodSource("provideTimestampToPaimonCases")
91+
void testToPaimonObjectForTimestamp(int precision, long serializedMicros, String expectedTs) {
92+
byte[] bytes = new byte[8];
93+
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).putLong(serializedMicros);
94+
95+
Timestamp actual =
96+
(Timestamp)
97+
IcebergConversions.toPaimonObject(DataTypes.TIMESTAMP(precision), bytes);
98+
99+
assertThat(actual.toString()).isEqualTo(expectedTs);
100+
}
101+
102+
private static Stream<Arguments> provideTimestampToPaimonCases() {
103+
return Stream.of(
104+
// Provide binary in micros; p=3..6 should all parse as micros
105+
Arguments.of(3, -1356022717123L, "1927-01-12T07:01:22.877"),
106+
Arguments.of(3, 1713790983524L, "2024-04-22T13:03:03.524"),
107+
Arguments.of(6, 1640690931207203L, "2021-12-28T11:28:51.207203"));
108+
}
109+
110+
@ParameterizedTest
111+
@MethodSource("provideInvalidTimestampCases")
112+
void testToPaimonObjectTimestampInvalid(int precision, long serializedMicros) {
113+
byte[] bytes = new byte[8];
114+
ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN).putLong(serializedMicros);
115+
116+
assertThatThrownBy(
117+
() ->
118+
IcebergConversions.toPaimonObject(
119+
DataTypes.TIMESTAMP(precision), bytes))
120+
.isInstanceOf(IllegalArgumentException.class)
121+
.hasMessageContaining(
122+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 6.");
123+
}
124+
125+
private static Stream<Arguments> provideInvalidTimestampCases() {
126+
return Stream.of(Arguments.of(0, 1698686153L), Arguments.of(9, 1698686153123456789L));
127+
}
128+
}

paimon-core/src/test/java/org/apache/paimon/iceberg/metadata/IcebergDataFieldTest.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -211,27 +211,27 @@ void testTimestampTypeConversions() {
211211
void testTimestampPrecisionValidation() {
212212
// Test invalid precision (<= 3)
213213
DataField invalidTimestampField =
214-
new DataField(1, "timestamp", new TimestampType(false, 3));
214+
new DataField(1, "timestamp", new TimestampType(false, 2));
215215
assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField))
216216
.isInstanceOf(IllegalArgumentException.class)
217217
.hasMessageContaining(
218-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9");
218+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 9");
219219

220220
// Test invalid precision (<= 3)
221221
DataField invalidTimestampField2 =
222-
new DataField(2, "timestamp", new TimestampType(false, 3));
222+
new DataField(2, "timestamp", new TimestampType(false, 2));
223223
assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField2))
224224
.isInstanceOf(IllegalArgumentException.class)
225225
.hasMessageContaining(
226-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9");
226+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 9");
227227

228228
// Test invalid local timezone timestamp precision (<= 3)
229229
DataField invalidTimestampLtzField =
230-
new DataField(3, "timestamptz", new LocalZonedTimestampType(false, 3));
230+
new DataField(3, "timestamptz", new LocalZonedTimestampType(false, 2));
231231
assertThatThrownBy(() -> new IcebergDataField(invalidTimestampLtzField))
232232
.isInstanceOf(IllegalArgumentException.class)
233233
.hasMessageContaining(
234-
"Paimon Iceberg compatibility only support timestamp type with precision from 4 to 9");
234+
"Paimon Iceberg compatibility only support timestamp type with precision from 3 to 9");
235235

236236
// Test valid precision boundaries
237237
DataField validTimestamp4 = new DataField(4, "timestamp", new TimestampType(false, 4));

0 commit comments

Comments
 (0)