This is an automated email from the ASF dual-hosted git repository.

lzljs3620320 pushed a commit to branch release-1.3
in repository https://gitbox.apache.org/repos/asf/paimon.git

commit d1767f50f7e14a8bbc4f3472a553650a1dc14ac8
Author: Jiajia Li <plusplusjia...@alibaba-inc.com>
AuthorDate: Thu Sep 18 09:58:29 2025 +0800

    [iceberg] Enhance Iceberg timestamp type compatibility with extended 
precision support. (#6278)
---
 docs/content/iceberg/overview.md                   | 44 ++++++++------
 .../paimon/iceberg/metadata/IcebergDataField.java  | 12 ++--
 .../iceberg/metadata/IcebergDataFieldTest.java     | 70 ++++++++++++++++++----
 3 files changed, 88 insertions(+), 38 deletions(-)

diff --git a/docs/content/iceberg/overview.md b/docs/content/iceberg/overview.md
index 8a953aa4a0..952bab4008 100644
--- a/docs/content/iceberg/overview.md
+++ b/docs/content/iceberg/overview.md
@@ -70,7 +70,7 @@ Set the following table options, so that Paimon tables can 
generate Iceberg comp
     </tbody>
 </table>
 
-For most SQL users, we recommend setting `'metadata.iceberg.storage' = 
'hadoop-catalog'`
+For most SQL users, we recommend setting `'metadata.iceberg.storage' = 
'hadoop-catalog'
 or `'metadata.iceberg.storage' = 'hive-catalog'`,
 so that all tables can be visited as an Iceberg warehouse.
 For Iceberg Java API users, you might consider setting 
`'metadata.iceberg.storage' = 'table-location'`,
@@ -84,22 +84,28 @@ If not set, the default behavior depends on the storage 
type.
 Paimon Iceberg compatibility currently supports the following data types.
 
 | Paimon Data Type | Iceberg Data Type |
-|------------------|-------------------|
-| `BOOLEAN`        | `boolean`         |
-| `INT`            | `int`             |
-| `BIGINT`         | `long`            |
-| `FLOAT`          | `float`           |
-| `DOUBLE`         | `double`          |
-| `DECIMAL`        | `decimal`         |
-| `CHAR`           | `string`          |
-| `VARCHAR`        | `string`          |
-| `BINARY`         | `binary`          |
-| `VARBINARY`      | `binary`          |
-| `DATE`           | `date`            |
-| `TIMESTAMP`*     | `timestamp`       |
-| `TIMESTAMP_LTZ`* | `timestamptz`     |
-| `ARRAY`          | `list`            |
-| `MAP`            | `map`             |
-| `ROW`            | `struct`          |
+|----------------|-------------------|
+| `BOOLEAN`      | `boolean`         |
+| `INT`          | `int`             |
+| `BIGINT`       | `long`            |
+| `FLOAT`        | `float`           |
+| `DOUBLE`       | `double`          |
+| `DECIMAL`      | `decimal`         |
+| `CHAR`         | `string`          |
+| `VARCHAR`      | `string`          |
+| `BINARY`       | `binary`          |
+| `VARBINARY`    | `binary`          |
+| `DATE`         | `date`            |
+| `TIMESTAMP` (precision 4-6)   | `timestamp`       |
+| `TIMESTAMP_LTZ` (precision 4-6) | `timestamptz`     |
+| `TIMESTAMP` (precision 7-9)  | `timestamp_ns`    |
+| `TIMESTAMP_LTZ` (precision 7-9) | `timestamptz_ns`  |
+| `ARRAY`        | `list`            |
+| `MAP`          | `map`             |
+| `ROW`          | `struct`          |
 
-*: `TIMESTAMP` and `TIMESTAMP_LTZ` type only support precision from 4 to 6
+{{< 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 7 to 9 use 
nanosecond precision and require Iceberg v3 format
+{{< /hint >}}
\ No newline at end of file
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 1a392b0ae9..3714d59bfb 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,15 +174,15 @@ public class IcebergDataField {
             case TIMESTAMP_WITHOUT_TIME_ZONE:
                 int timestampPrecision = ((TimestampType) 
dataType).getPrecision();
                 Preconditions.checkArgument(
-                        timestampPrecision > 3 && timestampPrecision <= 6,
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6.");
-                return "timestamp";
+                        timestampPrecision > 3 && timestampPrecision <= 9,
+                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 9.");
+                return timestampPrecision >= 7 ? "timestamp_ns" : "timestamp";
             case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
                 int timestampLtzPrecision = ((LocalZonedTimestampType) 
dataType).getPrecision();
                 Preconditions.checkArgument(
-                        timestampLtzPrecision > 3 && timestampLtzPrecision <= 
6,
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6.");
-                return "timestamptz";
+                        timestampLtzPrecision > 3 && timestampLtzPrecision <= 
9,
+                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 9.");
+                return timestampLtzPrecision >= 7 ? "timestamptz_ns" : 
"timestamptz";
             case ARRAY:
                 ArrayType arrayType = (ArrayType) dataType;
                 return new IcebergListType(
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 ce18ed2da1..f96a063459 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
@@ -171,6 +171,39 @@ class IcebergDataFieldTest {
                 new DataField(2, "timestamptz", new 
LocalZonedTimestampType(false, 6));
         IcebergDataField icebergTimestampLtz = new 
IcebergDataField(timestampLtzField);
         assertThat(icebergTimestampLtz.type()).isEqualTo("timestamptz");
+
+        // Test timestamp_ns (precision 7)
+        DataField timestampNs7Field = new DataField(3, "timestamp_ns", new 
TimestampType(false, 7));
+        IcebergDataField icebergTimestampNs7 = new 
IcebergDataField(timestampNs7Field);
+        assertThat(icebergTimestampNs7.type()).isEqualTo("timestamp_ns");
+
+        // Test timestamp_ns (precision 8)
+        DataField timestampNs8Field = new DataField(4, "timestamp_ns", new 
TimestampType(false, 8));
+        IcebergDataField icebergTimestampNs8 = new 
IcebergDataField(timestampNs8Field);
+        assertThat(icebergTimestampNs8.type()).isEqualTo("timestamp_ns");
+
+        // Test timestamp_ns (precision 9)
+        DataField timestampNs9Field = new DataField(5, "timestamp_ns", new 
TimestampType(false, 9));
+        IcebergDataField icebergTimestampNs9 = new 
IcebergDataField(timestampNs9Field);
+        assertThat(icebergTimestampNs9.type()).isEqualTo("timestamp_ns");
+
+        // Test timestamptz_ns (precision 7)
+        DataField timestampLtzNs7Field =
+                new DataField(6, "timestamptz_ns", new 
LocalZonedTimestampType(false, 7));
+        IcebergDataField icebergTimestampLtzNs7 = new 
IcebergDataField(timestampLtzNs7Field);
+        assertThat(icebergTimestampLtzNs7.type()).isEqualTo("timestamptz_ns");
+
+        // Test timestamptz_ns (precision 8)
+        DataField timestampLtzNs8Field =
+                new DataField(7, "timestamptz_ns", new 
LocalZonedTimestampType(false, 8));
+        IcebergDataField icebergTimestampLtzNs8 = new 
IcebergDataField(timestampLtzNs8Field);
+        assertThat(icebergTimestampLtzNs8.type()).isEqualTo("timestamptz_ns");
+
+        // Test timestamptz_ns (precision 9)
+        DataField timestampLtzNs9Field =
+                new DataField(8, "timestamptz_ns", new 
LocalZonedTimestampType(false, 9));
+        IcebergDataField icebergTimestampLtzNs9 = new 
IcebergDataField(timestampLtzNs9Field);
+        assertThat(icebergTimestampLtzNs9.type()).isEqualTo("timestamptz_ns");
     }
 
     @Test
@@ -182,15 +215,15 @@ class IcebergDataFieldTest {
         assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField))
                 .isInstanceOf(IllegalArgumentException.class)
                 .hasMessageContaining(
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6");
+                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 9");
 
-        // Test invalid precision (> 6)
+        // Test invalid precision (<= 3)
         DataField invalidTimestampField2 =
-                new DataField(2, "timestamp", new TimestampType(false, 7));
+                new DataField(2, "timestamp", new TimestampType(false, 3));
         assertThatThrownBy(() -> new IcebergDataField(invalidTimestampField2))
                 .isInstanceOf(IllegalArgumentException.class)
                 .hasMessageContaining(
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6");
+                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 9");
 
         // Test invalid local timezone timestamp precision (<= 3)
         DataField invalidTimestampLtzField =
@@ -198,15 +231,26 @@ class IcebergDataFieldTest {
         assertThatThrownBy(() -> new 
IcebergDataField(invalidTimestampLtzField))
                 .isInstanceOf(IllegalArgumentException.class)
                 .hasMessageContaining(
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6");
-
-        // Test invalid local timezone timestamp precision (> 6)
-        DataField invalidTimestampLtzField2 =
-                new DataField(4, "timestamptz", new 
LocalZonedTimestampType(false, 7));
-        assertThatThrownBy(() -> new 
IcebergDataField(invalidTimestampLtzField2))
-                .isInstanceOf(IllegalArgumentException.class)
-                .hasMessageContaining(
-                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 6");
+                        "Paimon Iceberg compatibility only support timestamp 
type with precision from 4 to 9");
+
+        // Test valid precision boundaries
+        DataField validTimestamp4 = new DataField(4, "timestamp", new 
TimestampType(false, 4));
+        IcebergDataField icebergTimestamp4 = new 
IcebergDataField(validTimestamp4);
+        assertThat(icebergTimestamp4.type()).isEqualTo("timestamp");
+
+        DataField validTimestamp9 = new DataField(5, "timestamp", new 
TimestampType(false, 9));
+        IcebergDataField icebergTimestamp9 = new 
IcebergDataField(validTimestamp9);
+        assertThat(icebergTimestamp9.type()).isEqualTo("timestamp_ns");
+
+        DataField validTimestampLtz4 =
+                new DataField(6, "timestamptz", new 
LocalZonedTimestampType(false, 4));
+        IcebergDataField icebergTimestampLtz4 = new 
IcebergDataField(validTimestampLtz4);
+        assertThat(icebergTimestampLtz4.type()).isEqualTo("timestamptz");
+
+        DataField validTimestampLtz9 =
+                new DataField(7, "timestamptz", new 
LocalZonedTimestampType(false, 9));
+        IcebergDataField icebergTimestampLtz9 = new 
IcebergDataField(validTimestampLtz9);
+        assertThat(icebergTimestampLtz9.type()).isEqualTo("timestamptz_ns");
     }
 
     @Test

Reply via email to