You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bo...@apache.org on 2022/04/21 12:50:22 UTC

[impala] branch master updated: IMPALA-10850: Interpret timestamp predicates in local timezone in IcebergScanNode

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

boroknagyz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git


The following commit(s) were added to refs/heads/master by this push:
     new e91c7810f IMPALA-10850: Interpret timestamp predicates in local timezone in IcebergScanNode
e91c7810f is described below

commit e91c7810f088245e9c21d591f63c56781e261572
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
AuthorDate: Mon Apr 11 16:54:40 2022 +0200

    IMPALA-10850: Interpret timestamp predicates in local timezone in IcebergScanNode
    
    IcebergScanNode interprets the timestamp literals as UTC timestamps
    during predicate pushdown to Iceberg. It causes problems when the
    Iceberg table uses TIMESTAMPTZ (which corresponds to TIMESTAMP WITH
    LOCAL TIME ZONE in SQL) because in the scanners we assume that the
    timestamp literals in a query are in local timezone.
    
    Hence, if the Iceberg table is partitioned by HOUR(ts), and Impala is
    running in a different timezone than UTC, then the following query
    doesn't return any rows:
    
     SELECT * from t
     WHERE ts = <some ts>;
    
    Because during predicate pushdown the timestamp is interpreted as a
    UTC timestamp (no conversion from local to UTC), but during query
    execution the timestamp data in the files are converted to local
    timezone, then compared to <some ts>. I.e. in the scanner the
    assumption is that <some ts> is in local timezone.
    
    On the other hand, when Iceberg type TIMESTAMP (which correcponds
    to TIMESTAMP WITHOUT TIME ZONE in SQL) is used, then we should just
    push down the timestamp values without any conversion. In this case
    there is no conversion in the scanners either.
    
    Testing:
     * added e2e test with TIMESTAMPTZ
     * added e2e test with TIMESTAMP
    
    Change-Id: I181be5d2fa004f69b457f69ff82dc2f9877f46fa
    Reviewed-on: http://gerrit.cloudera.org:8080/18399
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Csaba Ringhofer <cs...@cloudera.com>
---
 .../org/apache/impala/planner/IcebergScanNode.java |  34 +-
 .../org/apache/impala/planner/KuduScanNode.java    |   1 -
 testdata/data/README                               |  11 +
 ...1647fa77a-job_16493406300920_0023-00001.parquet | Bin 0 -> 664 bytes
 ...1647fa77a-job_16493406300920_0023-00003.parquet | Bin 0 -> 657 bytes
 ...1647fa77a-job_16493406300920_0023-00002.parquet | Bin 0 -> 657 bytes
 ...1647fa77a-job_16493406300920_0023-00004.parquet | Bin 0 -> 657 bytes
 ...1647fa77a-job_16493406300920_0023-00005.parquet | Bin 0 -> 657 bytes
 .../a366370e-6b9a-4698-82d0-95fb69b19afb-m0.avro   | Bin 0 -> 4518 bytes
 ...436-1-a366370e-6b9a-4698-82d0-95fb69b19afb.avro | Bin 0 -> 1896 bytes
 .../metadata/v1.metadata.json                      |  52 ++
 .../metadata/v2.metadata.json                      |  75 +++
 .../metadata/version-hint.text                     |   1 +
 ...c5a039891-job_16493406300920_0024-00001.parquet | Bin 0 -> 666 bytes
 ...c5a039891-job_16493406300920_0024-00004.parquet | Bin 0 -> 659 bytes
 ...c5a039891-job_16493406300920_0024-00002.parquet | Bin 0 -> 659 bytes
 ...c5a039891-job_16493406300920_0024-00003.parquet | Bin 0 -> 659 bytes
 ...c5a039891-job_16493406300920_0024-00005.parquet | Bin 0 -> 659 bytes
 ...c5a039891-job_16493406300920_0024-00006.parquet | Bin 0 -> 659 bytes
 .../94003077-eabb-4dab-95ec-52a1727ef853-m0.avro   | Bin 0 -> 4834 bytes
 ...437-1-94003077-eabb-4dab-95ec-52a1727ef853.avro | Bin 0 -> 1898 bytes
 .../metadata/v1.metadata.json                      |  52 ++
 .../metadata/v2.metadata.json                      |  75 +++
 .../metadata/version-hint.text                     |   1 +
 .../functional/functional_schema_template.sql      |  28 +
 .../datasets/functional/schema_constraints.csv     |   2 +
 .../queries/QueryTest/iceberg-query.test           | 635 +++++++++++++++++++++
 27 files changed, 961 insertions(+), 6 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java b/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
index 5e17b4d35..1a998b2d3 100644
--- a/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
@@ -26,9 +26,11 @@ import java.util.ListIterator;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.Expression.Operation;
 import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.types.Types;
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.BoolLiteral;
@@ -46,16 +48,19 @@ import org.apache.impala.catalog.FeCatalogUtils;
 import org.apache.impala.catalog.FeFsPartition;
 import org.apache.impala.catalog.FeFsTable;
 import org.apache.impala.catalog.FeIcebergTable;
+import org.apache.impala.catalog.IcebergColumn;
 import org.apache.impala.catalog.TableLoadingException;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
+import org.apache.impala.common.InternalException;
 import org.apache.impala.common.Pair;
 import org.apache.impala.util.IcebergUtil;
+import org.apache.impala.util.ExprUtil;
 
 import com.google.common.base.Preconditions;
-import org.apache.impala.util.ExprUtil;
+
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -205,7 +210,9 @@ public class IcebergScanNode extends HdfsScanNode {
     // If predicate contains map/struct, this column would be null
     if (ref.getDesc().getColumn() == null) return false;
 
-    String colName = ref.getDesc().getColumn().getName();
+    IcebergColumn iceCol = (IcebergColumn)ref.getDesc().getColumn();
+    Schema iceSchema = icebergTable_.getIcebergSchema();
+    String colName = iceCol.getName();
     UnboundPredicate unboundPredicate = null;
     switch (literal.getType().getPrimitiveType()) {
       case BOOLEAN: {
@@ -243,9 +250,26 @@ public class IcebergScanNode extends HdfsScanNode {
         break;
       }
       case TIMESTAMP: {
-        // TODO(IMPALA-10850): interpret timestamps in local timezone.
-        long unixMicros = ExprUtil.utcTimestampToUnixTimeMicros(analyzer, literal);
-        unboundPredicate = Expressions.predicate(op, colName, unixMicros);
+        try {
+          org.apache.iceberg.types.Type iceType = iceSchema.findType(iceCol.getFieldId());
+          Preconditions.checkState(iceType instanceof Types.TimestampType);
+          Types.TimestampType tsType = (Types.TimestampType)iceType;
+          long unixMicros = 0;
+          if (tsType.shouldAdjustToUTC()) {
+            unixMicros = ExprUtil.localTimestampToUnixTimeMicros(analyzer, literal);
+          } else {
+            unixMicros = ExprUtil.utcTimestampToUnixTimeMicros(analyzer, literal);
+          }
+          unboundPredicate = Expressions.predicate(op, colName, unixMicros);
+        } catch (InternalException ex) {
+          // We cannot interpret the timestamp literal. Maybe the timestamp is invalid,
+          // or the local timestamp ambigously converts to UTC due to daylight saving
+          // time backward turn. E.g. '2021-10-31 02:15:00 Europe/Budapest' converts to
+          // either '2021-10-31 00:15:00 UTC' or '2021-10-31 01:15:00 UTC'.
+          LOG.warn("Exception occurred during timestamp conversion: " + ex.toString() +
+              "\nThis means timestamp predicate is not pushed to Iceberg, let Impala " +
+              "backend handle it.");
+        }
         break;
       }
       case DATE: {
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index d0d106a95..6663460ae 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -559,7 +559,6 @@ public class KuduScanNode extends ScanNode {
       case TIMESTAMP: {
         try {
           // TODO: Simplify when Impala supports a 64-bit TIMESTAMP type.
-          // TODO(IMPALA-10850): interpret timestamps in local timezone.
           kuduPredicate = KuduPredicate.newComparisonPredicate(column, op,
               ExprUtil.utcTimestampToUnixTimeMicros(analyzer, literal));
         } catch (Exception e) {
diff --git a/testdata/data/README b/testdata/data/README
index 570c1e87a..568152329 100644
--- a/testdata/data/README
+++ b/testdata/data/README
@@ -716,6 +716,17 @@ The tables that have the following schema changes since table migration:
 * Partition DECIMAL(5,3) column to DECIMAL(8,3)
 * Non-partition column has been moved to end of the schema
 
+iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part:
+Written by Hive, contains Iceberg TIMESTAMP type and the table is partitioned by HOUR(ts).
+create table iceberg_timestamp_part (i int, ts timestamp) partitioned by spec (hour(ts))  stored by iceberg;
+insert into iceberg_timestamp_part values (-2, '1969-01-01 01:00:00'), (-1, '1969-01-01 01:15:00'), (1, '2021-10-31 02:15:00'), (2, '2021-01-10 12:00:00'), (3, '2022-04-11 00:04:00'), (4, '2022-04-11 12:04:55');
+
+iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part:
+Written by Hive, contains Iceberg TIMESTAMPTZ type and the table is partitioned by
+HOUR(ts). The local timezone was 'Europe/Budapest';
+create table iceberg_timestamptz_part (i int, ts timestamp with local time zone) partitioned by spec (hour(ts))  stored by iceberg;
+insert into iceberg_timestamptz_part values (-2, '1969-01-01 01:00:00'), (-1, '1969-01-01 01:15:00'), (0, '2021-10-31 00:15:00 UTC'), (1, '2021-10-31 01:15:00 UTC'), (2, '2021-01-10 12:00:00'), (3, '2022-04-11 00:04:00'), (4, '2022-04-11 12:04:55');
+
 iceberg_test/hadoop_catalog/ice/iceberg_uppercase_col:
 Generated by Impala, then modified the metadata.json file to contain uppercase characters.
 
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=1969-01-01-01/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00001.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=1969-01-01-01/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00001.parquet
new file mode 100644
index 000000000..27a7c63bc
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=1969-01-01-01/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00001.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-01-10-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00003.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-01-10-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00003.parquet
new file mode 100644
index 000000000..f39fae8b1
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-01-10-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00003.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-10-31-02/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00002.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-10-31-02/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00002.parquet
new file mode 100644
index 000000000..7beba5e7b
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2021-10-31-02/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00002.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-00/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00004.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-00/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00004.parquet
new file mode 100644
index 000000000..2ee7adf42
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-00/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00004.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00005.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00005.parquet
new file mode 100644
index 000000000..096bddeec
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/data/ts_hour=2022-04-11-12/00000-0-boroknagyz_20220419181820_3b0f79ee-1aff-4983-98cf-7d01647fa77a-job_16493406300920_0023-00005.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/a366370e-6b9a-4698-82d0-95fb69b19afb-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/a366370e-6b9a-4698-82d0-95fb69b19afb-m0.avro
new file mode 100644
index 000000000..ce64f3159
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/a366370e-6b9a-4698-82d0-95fb69b19afb-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/snap-1967339514069250436-1-a366370e-6b9a-4698-82d0-95fb69b19afb.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/snap-1967339514069250436-1-a366370e-6b9a-4698-82d0-95fb69b19afb.avro
new file mode 100644
index 000000000..8ac0fb1ad
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/snap-1967339514069250436-1-a366370e-6b9a-4698-82d0-95fb69b19afb.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v1.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v1.metadata.json
new file mode 100644
index 000000000..a09f1b59d
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v1.metadata.json
@@ -0,0 +1,52 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "551caed0-597f-43d8-9cf3-660b8371da67",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part",
+  "last-updated-ms" : 1650385008403,
+  "last-column-id" : 2,
+  "schema" : {
+    "type" : "struct",
+    "fields" : [ {
+      "id" : 1,
+      "name" : "i",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "ts",
+      "required" : false,
+      "type" : "timestamp"
+    } ]
+  },
+  "partition-spec" : [ {
+    "name" : "ts_hour",
+    "transform" : "hour",
+    "source-id" : 2,
+    "field-id" : 1000
+  } ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ {
+      "name" : "ts_hour",
+      "transform" : "hour",
+      "source-id" : 2,
+      "field-id" : 1000
+    } ]
+  } ],
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v2.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v2.metadata.json
new file mode 100644
index 000000000..a9c65f99c
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/v2.metadata.json
@@ -0,0 +1,75 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "551caed0-597f-43d8-9cf3-660b8371da67",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part",
+  "last-updated-ms" : 1650385111326,
+  "last-column-id" : 2,
+  "schema" : {
+    "type" : "struct",
+    "fields" : [ {
+      "id" : 1,
+      "name" : "i",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "ts",
+      "required" : false,
+      "type" : "timestamp"
+    } ]
+  },
+  "partition-spec" : [ {
+    "name" : "ts_hour",
+    "transform" : "hour",
+    "source-id" : 2,
+    "field-id" : 1000
+  } ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ {
+      "name" : "ts_hour",
+      "transform" : "hour",
+      "source-id" : 2,
+      "field-id" : 1000
+    } ]
+  } ],
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 1967339514069250436,
+  "snapshots" : [ {
+    "snapshot-id" : 1967339514069250436,
+    "timestamp-ms" : 1650385111326,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "5",
+      "added-records" : "6",
+      "added-files-size" : "3292",
+      "changed-partition-count" : "5",
+      "total-records" : "6",
+      "total-files-size" : "3292",
+      "total-data-files" : "5",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/snap-1967339514069250436-1-a366370e-6b9a-4698-82d0-95fb69b19afb.avro"
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1650385111326,
+    "snapshot-id" : 1967339514069250436
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1650385008403,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/00000-569c3f8a-5117-4bbf-970d-96eed4df0b4b.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/version-hint.text b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/version-hint.text
new file mode 100644
index 000000000..0cfbf0888
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part/metadata/version-hint.text
@@ -0,0 +1 @@
+2
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=1969-01-01-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00001.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=1969-01-01-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00001.parquet
new file mode 100644
index 000000000..8f439f993
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=1969-01-01-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00001.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-01-10-11/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00004.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-01-10-11/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00004.parquet
new file mode 100644
index 000000000..938b5cde0
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-01-10-11/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00004.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00002.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00002.parquet
new file mode 100644
index 000000000..9fc7fe842
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-00/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00002.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-01/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00003.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-01/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00003.parquet
new file mode 100644
index 000000000..1123ab202
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2021-10-31-01/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00003.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-10-22/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00005.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-10-22/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00005.parquet
new file mode 100644
index 000000000..3efa7dd66
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-10-22/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00005.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-11-10/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00006.parquet b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-11-10/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00006.parquet
new file mode 100644
index 000000000..81c465849
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/data/ts_hour=2022-04-11-10/00000-0-boroknagyz_20220419182502_45a45ed8-85ff-4046-b834-648c5a039891-job_16493406300920_0024-00006.parquet differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/94003077-eabb-4dab-95ec-52a1727ef853-m0.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/94003077-eabb-4dab-95ec-52a1727ef853-m0.avro
new file mode 100644
index 000000000..7dfe49622
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/94003077-eabb-4dab-95ec-52a1727ef853-m0.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/snap-2778998487482282437-1-94003077-eabb-4dab-95ec-52a1727ef853.avro b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/snap-2778998487482282437-1-94003077-eabb-4dab-95ec-52a1727ef853.avro
new file mode 100644
index 000000000..b42de09f6
Binary files /dev/null and b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/snap-2778998487482282437-1-94003077-eabb-4dab-95ec-52a1727ef853.avro differ
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v1.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v1.metadata.json
new file mode 100644
index 000000000..543ba6010
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v1.metadata.json
@@ -0,0 +1,52 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "02bd0277-e602-4880-8bac-45647525af99",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part",
+  "last-updated-ms" : 1650385496273,
+  "last-column-id" : 2,
+  "schema" : {
+    "type" : "struct",
+    "fields" : [ {
+      "id" : 1,
+      "name" : "i",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "ts",
+      "required" : false,
+      "type" : "timestamptz"
+    } ]
+  },
+  "partition-spec" : [ {
+    "name" : "ts_hour",
+    "transform" : "hour",
+    "source-id" : 2,
+    "field-id" : 1000
+  } ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ {
+      "name" : "ts_hour",
+      "transform" : "hour",
+      "source-id" : 2,
+      "field-id" : 1000
+    } ]
+  } ],
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : -1,
+  "snapshots" : [ ],
+  "snapshot-log" : [ ],
+  "metadata-log" : [ ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v2.metadata.json b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v2.metadata.json
new file mode 100644
index 000000000..0da65bb8a
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/v2.metadata.json
@@ -0,0 +1,75 @@
+{
+  "format-version" : 1,
+  "table-uuid" : "02bd0277-e602-4880-8bac-45647525af99",
+  "location" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part",
+  "last-updated-ms" : 1650385512100,
+  "last-column-id" : 2,
+  "schema" : {
+    "type" : "struct",
+    "fields" : [ {
+      "id" : 1,
+      "name" : "i",
+      "required" : false,
+      "type" : "int"
+    }, {
+      "id" : 2,
+      "name" : "ts",
+      "required" : false,
+      "type" : "timestamptz"
+    } ]
+  },
+  "partition-spec" : [ {
+    "name" : "ts_hour",
+    "transform" : "hour",
+    "source-id" : 2,
+    "field-id" : 1000
+  } ],
+  "default-spec-id" : 0,
+  "partition-specs" : [ {
+    "spec-id" : 0,
+    "fields" : [ {
+      "name" : "ts_hour",
+      "transform" : "hour",
+      "source-id" : 2,
+      "field-id" : 1000
+    } ]
+  } ],
+  "default-sort-order-id" : 0,
+  "sort-orders" : [ {
+    "order-id" : 0,
+    "fields" : [ ]
+  } ],
+  "properties" : {
+    "engine.hive.enabled" : "true",
+    "bucketing_version" : "2",
+    "serialization.format" : "1",
+    "storage_handler" : "org.apache.iceberg.mr.hive.HiveIcebergStorageHandler"
+  },
+  "current-snapshot-id" : 2778998487482282437,
+  "snapshots" : [ {
+    "snapshot-id" : 2778998487482282437,
+    "timestamp-ms" : 1650385512100,
+    "summary" : {
+      "operation" : "append",
+      "added-data-files" : "6",
+      "added-records" : "7",
+      "added-files-size" : "3961",
+      "changed-partition-count" : "6",
+      "total-records" : "7",
+      "total-files-size" : "3961",
+      "total-data-files" : "6",
+      "total-delete-files" : "0",
+      "total-position-deletes" : "0",
+      "total-equality-deletes" : "0"
+    },
+    "manifest-list" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/snap-2778998487482282437-1-94003077-eabb-4dab-95ec-52a1727ef853.avro"
+  } ],
+  "snapshot-log" : [ {
+    "timestamp-ms" : 1650385512100,
+    "snapshot-id" : 2778998487482282437
+  } ],
+  "metadata-log" : [ {
+    "timestamp-ms" : 1650385496273,
+    "metadata-file" : "/test-warehouse/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/00000-3571399d-7765-4431-82fe-ba9f837f24d3.metadata.json"
+  } ]
+}
\ No newline at end of file
diff --git a/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/version-hint.text b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/version-hint.text
new file mode 100644
index 000000000..0cfbf0888
--- /dev/null
+++ b/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part/metadata/version-hint.text
@@ -0,0 +1 @@
+2
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index 045c57bcb..996591da2 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -3297,6 +3297,34 @@ STORED AS ICEBERG;
 ---- DATASET
 functional
 ---- BASE_TABLE_NAME
+iceberg_timestamp_part
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS ICEBERG
+TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
+              'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
+              'iceberg.table_identifier'='ice.iceberg_timestamp_part');
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
+hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamp_part /test-warehouse/iceberg_test/hadoop_catalog/ice
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
+iceberg_timestamptz_part
+---- CREATE
+CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
+STORED AS ICEBERG
+TBLPROPERTIES('write.format.default'='parquet', 'iceberg.catalog'='hadoop.catalog',
+              'iceberg.catalog_location'='/test-warehouse/iceberg_test/hadoop_catalog',
+              'iceberg.table_identifier'='ice.iceberg_timestamptz_part');
+---- DEPENDENT_LOAD
+`hadoop fs -mkdir -p /test-warehouse/iceberg_test/hadoop_catalog/ice && \
+hadoop fs -put -f ${IMPALA_HOME}/testdata/data/iceberg_test/hadoop_catalog/ice/iceberg_timestamptz_part /test-warehouse/iceberg_test/hadoop_catalog/ice
+====
+---- DATASET
+functional
+---- BASE_TABLE_NAME
 iceberg_uppercase_col
 ---- CREATE
 CREATE EXTERNAL TABLE IF NOT EXISTS {db_name}{db_suffix}.{table_name}
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index c814e4db5..6b26d3ccf 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -77,6 +77,8 @@ table_name:iceberg_alltypes_part, constraint:restrict_to, table_format:parquet/n
 table_name:iceberg_alltypes_part_orc, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_legacy_partition_schema_evolution, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_legacy_partition_schema_evolution_orc, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_timestamp_part, constraint:restrict_to, table_format:parquet/none/none
+table_name:iceberg_timestamptz_part, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_uppercase_col, constraint:restrict_to, table_format:parquet/none/none
 table_name:iceberg_v2_delete_positional, constraint:restrict_to, table_format:parquet/none/none
 
diff --git a/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test b/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
index 8adb1d8c1..f853ead68 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/iceberg-query.test
@@ -521,3 +521,638 @@ SELECT * FROM iceberg_uppercase_col;
 ---- TYPES
 STRING
 ====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+order by i;
+---- RESULTS
+-2,1969-01-01 00:00:00
+-1,1969-01-01 00:15:00
+0,2021-10-31 00:15:00
+1,2021-10-31 01:15:00
+2,2021-01-10 11:00:00
+3,2022-04-10 22:04:00
+4,2022-04-11 10:04:55
+---- TYPES
+INT, TIMESTAMP
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+0,2021-10-31 02:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '1969-01-01 00:00:00';
+---- RESULTS
+-2,1969-01-01 00:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '1969-01-01 00:15:00';
+---- RESULTS
+-1,1969-01-01 00:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2021-10-31 00:15:00';
+---- RESULTS
+0,2021-10-31 00:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2021-10-31 01:15:00';
+---- RESULTS
+1,2021-10-31 01:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2021-01-10 11:00:00';
+---- RESULTS
+2,2021-01-10 11:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2022-04-10 22:04:00';
+---- RESULTS
+3,2022-04-10 22:04:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2022-04-11 10:04:55';
+---- RESULTS
+4,2022-04-11 10:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts < '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 00:00:00
+-1,1969-01-01 00:15:00
+0,2021-10-31 00:15:00
+2,2021-01-10 11:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 3
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts > '1969-01-01 00:00:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 00:15:00
+0,2021-10-31 00:15:00
+1,2021-10-31 01:15:00
+2,2021-01-10 11:00:00
+3,2022-04-10 22:04:00
+4,2022-04-11 10:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts > '1969-01-01 00:15:00' and  ts < '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+0,2021-10-31 00:15:00
+2,2021-01-10 11:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts >= '1969-01-01 00:15:00' and  ts <= '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 00:15:00
+0,2021-10-31 00:15:00
+1,2021-10-31 01:15:00
+2,2021-01-10 11:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 4
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts between '1969-01-01 00:15:00' and '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 00:15:00
+0,2021-10-31 00:15:00
+1,2021-10-31 01:15:00
+2,2021-01-10 11:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 4
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '1969-01-01 01:00:00';
+---- RESULTS
+-2,1969-01-01 01:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '1969-01-01 01:15:00';
+---- RESULTS
+-1,1969-01-01 01:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+# Query ambigous timestamp. '2021-10-31 02:15:00' converts
+# to '2021-10-31 00:15:00 UTC' and '2021-10-31 01:15:00 UTC' due to DST.
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2021-10-31 02:15:00';
+---- RESULTS
+0,2021-10-31 02:15:00
+1,2021-10-31 02:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+aggregation(SUM, NumStatsFilteredRowGroups): 4
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2021-01-10 12:00:00';
+---- RESULTS
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2022-04-11 00:04:00';
+---- RESULTS
+3,2022-04-11 00:04:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts = '2022-04-11 12:04:55';
+---- RESULTS
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts < '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts > '1969-01-01 01:00:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 01:15:00
+0,2021-10-31 02:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts > '1969-01-01 00:15:00' and  ts < '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts >= '1969-01-01 00:15:00' and  ts <= '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+0,2021-10-31 02:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamptz_part
+WHERE ts between '1969-01-01 00:15:00' and '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+0,2021-10-31 02:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 6
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '1969-01-01 01:00:00';
+---- RESULTS
+-2,1969-01-01 01:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '1969-01-01 01:15:00';
+---- RESULTS
+-1,1969-01-01 01:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2021-10-31 02:15:00';
+---- RESULTS
+1,2021-10-31 02:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2021-01-10 12:00:00';
+---- RESULTS
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2022-04-11 00:04:00';
+---- RESULTS
+3,2022-04-11 00:04:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2022-04-11 12:04:55';
+---- RESULTS
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts < '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts > '1969-01-01 01:00:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 5
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts > '1969-01-01 00:15:00' and  ts < '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts >= '1969-01-01 00:15:00' and  ts <= '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='UTC';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts between '1969-01-01 00:15:00' and '2021-10-31 01:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '1969-01-01 01:00:00';
+---- RESULTS
+-2,1969-01-01 01:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '1969-01-01 01:15:00';
+---- RESULTS
+-1,1969-01-01 01:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2021-10-31 02:15:00';
+---- RESULTS
+1,2021-10-31 02:15:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2021-01-10 12:00:00';
+---- RESULTS
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2022-04-11 00:04:00';
+---- RESULTS
+3,2022-04-11 00:04:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts = '2022-04-11 12:04:55';
+---- RESULTS
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 1
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts < '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts > '1969-01-01 01:00:00'
+order by i;
+---- RESULTS
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+3,2022-04-11 00:04:00
+4,2022-04-11 12:04:55
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 5
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts > '1969-01-01 00:15:00' and  ts < '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 2
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts >= '1969-01-01 00:15:00' and  ts <= '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 3
+====
+---- QUERY
+SET TIMEZONE='Europe/Budapest';
+SELECT * FROM iceberg_timestamp_part
+WHERE ts between '1969-01-01 00:15:00' and '2021-10-31 02:15:00'
+order by i;
+---- RESULTS
+-2,1969-01-01 01:00:00
+-1,1969-01-01 01:15:00
+1,2021-10-31 02:15:00
+2,2021-01-10 12:00:00
+---- TYPES
+INT, TIMESTAMP
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 3
+====