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 2fffac3  IMPALA-11175: Iceberg table cannot be loaded when partition 
value is NULL
2fffac3 is described below

commit 2fffac3bad3b86d8aa6d6c1c31400578373de505
Author: Zoltan Borok-Nagy <borokna...@cloudera.com>
AuthorDate: Thu Mar 10 20:53:17 2022 +0100

    IMPALA-11175: Iceberg table cannot be loaded when partition value is NULL
    
    When Impala created the metadata objects about the Iceberg data files it
    tried to convert the partition values to strings. But the partition
    values can be NULLs as well. The code didn't expect this, so we got a
    NullPointerException.
    
    With this patch we pass the table's null partition key value in case
    of NULLs.
    
    Testing:
     * added e2e tests
    
    Change-Id: I88c4f7a2c2db4f6390c8ee5c08baddc96b04602e
    Reviewed-on: http://gerrit.cloudera.org:8080/18307
    Reviewed-by: Tamas Mate <tma...@apache.org>
    Reviewed-by: Gabor Kaszab <gaborkas...@cloudera.com>
    Tested-by: Impala Public Jenkins <impala-public-jenk...@cloudera.com>
---
 be/src/exec/file-metadata-utils.cc                 |  8 ++-
 .../org/apache/impala/catalog/FeIcebergTable.java  |  5 +-
 .../org/apache/impala/planner/IcebergScanNode.java |  8 +--
 .../java/org/apache/impala/util/IcebergUtil.java   | 31 +++++----
 .../iceberg-partition-transform-insert.test        | 73 ++++++++++++++++++----
 .../QueryTest/iceberg-partitioned-insert.test      | 19 ++++++
 6 files changed, 112 insertions(+), 32 deletions(-)

diff --git a/be/src/exec/file-metadata-utils.cc 
b/be/src/exec/file-metadata-utils.cc
index 7ed2b23..708b4f5 100644
--- a/be/src/exec/file-metadata-utils.cc
+++ b/be/src/exec/file-metadata-utils.cc
@@ -52,7 +52,7 @@ Tuple* FileMetadataUtils::CreateTemplateTuple(MemPool* 
mem_pool) {
   }
   using namespace org::apache::impala::fb;
   TextConverter text_converter(/* escape_char = */ '\\',
-      scan_node_->hdfs_table()->null_column_value(),
+      scan_node_->hdfs_table()->null_partition_key_value(),
       /* check_null = */ true, /* strict_mode = */ true);
   const FbFileMetadata* file_metadata = file_desc_->file_metadata;
   const FbIcebergMetadata* ice_metadata = file_metadata->iceberg_metadata();
@@ -83,9 +83,11 @@ Tuple* FileMetadataUtils::CreateTemplateTuple(MemPool* 
mem_pool) {
                                     mem_pool)) {
         ErrorMsg error_msg(TErrorCode::GENERAL,
             Substitute("Could not parse partition value for "
-                "column '$0' in file '$1'. Partition string is '$2'",
+                "column '$0' in file '$1'. Partition string is '$2' "
+                "NULL Partition key value is '$3'",
                 col_desc.name(), file_desc_->filename,
-                transform->transform_value()->c_str()));
+                transform->transform_value()->c_str(),
+                scan_node_->hdfs_table()->null_partition_key_value()));
         // Dates are stored as INTs in the partition data in Iceberg, so let's 
try
         // to parse them as INTs.
         if (col_desc.type().type == PrimitiveType::TYPE_DATE) {
diff --git a/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java 
b/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
index 7473346..db1249f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeIcebergTable.java
@@ -524,7 +524,6 @@ public interface FeIcebergTable extends FeFsTable {
       Map<String, HdfsPartition.FileDescriptor> fileDescMap = new HashMap<>();
       List<DataFile> dataFileList = IcebergUtil.getIcebergDataFiles(table,
           new ArrayList<>(), /*timeTravelSpecl=*/null);
-      Table iceTable = IcebergUtil.loadTable(table);
       for (DataFile dataFile : dataFileList) {
           Path path = new Path(dataFile.path().toString());
           if (hdfsFileDescMap.containsKey(path.toUri().getPath())) {
@@ -532,7 +531,7 @@ public interface FeIcebergTable extends FeFsTable {
             HdfsPartition.FileDescriptor fsFd = hdfsFileDescMap.get(
                 path.toUri().getPath());
             HdfsPartition.FileDescriptor iceFd = fsFd.cloneWithFileMetadata(
-                IcebergUtil.createIcebergMetadata(iceTable, dataFile));
+                IcebergUtil.createIcebergMetadata(table, dataFile));
             fileDescMap.put(pathHash, iceFd);
           } else {
             LOG.warn("Iceberg DataFile '{}' cannot be found in the HDFS 
recursive file "
@@ -541,7 +540,7 @@ public interface FeIcebergTable extends FeFsTable {
                 new Path(dataFile.path().toString()),
                 new Path(table.getIcebergTableLocation()), 
table.getHostIndex());
             HdfsPartition.FileDescriptor iceFd = 
fileDesc.cloneWithFileMetadata(
-                IcebergUtil.createIcebergMetadata(iceTable, dataFile));
+                IcebergUtil.createIcebergMetadata(table, dataFile));
             fileDescMap.put(IcebergUtil.getDataFilePathHash(dataFile), iceFd);
           }
       }
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 7330149..4ab98b7 100644
--- a/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/IcebergScanNode.java
@@ -139,12 +139,12 @@ public class IcebergScanNode extends HdfsScanNode {
         }
         // Add file descriptor to the cache.
         try {
-          org.apache.iceberg.Table iceTable = 
IcebergUtil.loadTable(icebergTable_);
           fileDesc = fileDesc.cloneWithFileMetadata(
-              IcebergUtil.createIcebergMetadata(iceTable, dataFile));
-        } catch (TableLoadingException ex) {
+              IcebergUtil.createIcebergMetadata(icebergTable_, dataFile));
+        } catch (TableLoadingException e) {
+          // TODO: get rid of try-catch TableLoadingException once we have 
IMPALA-10737.
           throw new ImpalaRuntimeException(String.format(
-              "Cannot load Iceberg table for: %s", 
icebergTable_.getFullName()), ex);
+              "Failed to load Iceberg table: %s", 
icebergTable_.getFullName()), e);
         }
         icebergTable_.getPathHashToFileDescMap().put(
             IcebergUtil.getDataFilePathHash(dataFile), fileDesc);
diff --git a/fe/src/main/java/org/apache/impala/util/IcebergUtil.java 
b/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
index a7f184c..8df4fe7 100644
--- a/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/IcebergUtil.java
@@ -865,9 +865,10 @@ public class IcebergUtil {
    * It creates a flatbuffer so it can be passed between machines and 
processes without
    * further de/serialization.
    */
-  public static FbFileMetadata createIcebergMetadata(Table iceTbl, DataFile 
df) {
+  public static FbFileMetadata createIcebergMetadata(FeIcebergTable feTbl, 
DataFile df)
+      throws TableLoadingException {
     FlatBufferBuilder fbb = new FlatBufferBuilder(1);
-    int iceOffset = createIcebergMetadata(fbb, iceTbl, df);
+    int iceOffset = createIcebergMetadata(feTbl, fbb, df);
     fbb.finish(FbFileMetadata.createFbFileMetadata(fbb, iceOffset));
     ByteBuffer bb = fbb.dataBuffer().slice();
     ByteBuffer compressedBb = ByteBuffer.allocate(bb.capacity());
@@ -875,12 +876,14 @@ public class IcebergUtil {
     return 
FbFileMetadata.getRootAsFbFileMetadata((ByteBuffer)compressedBb.flip());
   }
 
-  private static int createIcebergMetadata(FlatBufferBuilder fbb, Table iceTbl,
-      DataFile df) {
+  private static int createIcebergMetadata(FeIcebergTable feTbl, 
FlatBufferBuilder fbb,
+      DataFile df) throws TableLoadingException {
+    //TODO: avoid loading the table once we have IMPALA-10737 again:
+    Table iceTbl = loadTable(feTbl);
     int partKeysOffset = -1;
     PartitionSpec spec = iceTbl.specs().get(df.specId());
     if (spec != null && !spec.fields().isEmpty()) {
-      partKeysOffset = createPartitionKeys(fbb, spec, df);
+      partKeysOffset = createPartitionKeys(feTbl, fbb, spec, df);
     }
     FbIcebergMetadata.startFbIcebergMetadata(fbb);
     byte fileFormat = -1;
@@ -896,25 +899,31 @@ public class IcebergUtil {
     return FbIcebergMetadata.endFbIcebergMetadata(fbb);
   }
 
-  private static int createPartitionKeys(FlatBufferBuilder fbb, PartitionSpec 
spec,
-      DataFile df) {
+  private static int createPartitionKeys(FeIcebergTable feTbl, 
FlatBufferBuilder fbb,
+      PartitionSpec spec, DataFile df) {
     Preconditions.checkState(spec.fields().size() == df.partition().size());
     int[] partitionKeyOffsets = new int[spec.fields().size()];
     for (int i = 0; i < spec.fields().size(); ++i) {
       partitionKeyOffsets[i] =
-          createPartitionTransformValue(fbb, spec, df, i);
+          createPartitionTransformValue(feTbl, fbb, spec, df, i);
     }
     return FbIcebergMetadata.createPartitionKeysVector(fbb, 
partitionKeyOffsets);
   }
 
-  private static int createPartitionTransformValue(FlatBufferBuilder fbb,
-      PartitionSpec spec, DataFile df, int fieldIndex) {
+  private static int createPartitionTransformValue(FeIcebergTable feTbl,
+      FlatBufferBuilder fbb, PartitionSpec spec, DataFile df, int fieldIndex) {
     PartitionField field = spec.fields().get(fieldIndex);
     Pair<Byte, Integer> transform = getFbTransform(spec.schema(), field);
     int valueOffset = -1;
     if (transform.first != FbIcebergTransformType.VOID) {
       Object partValue = df.partition().get(fieldIndex, Object.class);
-      valueOffset = fbb.createString(partValue.toString());
+      String partValueString;
+      if (partValue != null) {
+        partValueString = partValue.toString();
+      } else {
+        partValueString = feTbl.getNullPartitionKeyValue();
+      }
+      valueOffset = fbb.createString(partValueString);
     }
     
FbIcebergPartitionTransformValue.startFbIcebergPartitionTransformValue(fbb);
     FbIcebergPartitionTransformValue.addTransformType(fbb, transform.first);
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-partition-transform-insert.test
 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-partition-transform-insert.test
index 01a0513..460f5b9 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-partition-transform-insert.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-partition-transform-insert.test
@@ -7,7 +7,7 @@ partitioned by spec (bucket(7, s))
 stored as iceberg;
 ====
 ---- QUERY
-insert into single_col_bucket values ('a'), ('b'), ('c'), ('d'), ('e'), ('f');
+insert into single_col_bucket values ('a'), ('b'), ('c'), ('d'), ('e'), ('f'), 
(NULL);
 select * from single_col_bucket;
 ---- RESULTS
 'a'
@@ -16,6 +16,7 @@ select * from single_col_bucket;
 'd'
 'e'
 'f'
+'NULL'
 ---- TYPES
 STRING
 ====
@@ -26,6 +27,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_bucket/data/s_bucket
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_bucket/data/s_bucket=2/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_bucket/data/s_bucket=4/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_bucket/data/s_bucket=6/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_bucket/data/s_bucket=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -153,7 +155,7 @@ stored as iceberg;
 ---- QUERY
 insert into single_col_truncate
 values (1.1234), (10.5432), (200.786), (314.1592), (98765.4321),
-       (1.1235), (10.5433), (200.789), (314.1598), (98765.43);
+       (1.1235), (10.5433), (200.789), (314.1598), (98765.43), (NULL);
 select * from single_col_truncate;
 ---- RESULTS
 1.1234
@@ -166,6 +168,7 @@ select * from single_col_truncate;
 314.1598
 98765.4300
 98765.4321
+NULL
 ---- TYPES
 DECIMAL
 ====
@@ -177,6 +180,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_truncate/data/d_trun
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_truncate/data/d_trunc=200.7800/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_truncate/data/d_trunc=314.1500/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_truncate/data/d_trunc=98765.4300/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/single_col_truncate/data/d_trunc=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -236,7 +240,8 @@ insert into multi_col_truncate values
 (2, 222, 421,            'the quick brown fox jumps over the lazy dog'),
 (6, 333, 113211.2,       'the quick brown fox over the lazy dog the quick'),
 (7, 444, 1111154.11432,  'the quick fox brown jump impala over the quick fox 
brown the over jump'),
-(18, 555, 9999913.232432,'the quick impala fox dog parrot lion');
+(18, 555, 9999913.232432,'the quick impala fox dog parrot lion'),
+(NULL, NULL, NULL, NULL);
 select * from multi_col_truncate;
 ---- RESULTS
 1,11,11111.111111,'the quick brown fox jumps over the lazy dog'
@@ -244,6 +249,7 @@ select * from multi_col_truncate;
 6,333,113211.200000,'the quick brown fox over the lazy dog the quick'
 7,444,1111154.114320,'the quick fox brown jump impala over the quick fox brown 
the over jump'
 18,555,9999913.232432,'the quick impala fox dog parrot lion'
+NULL,NULL,NULL,'NULL'
 ---- TYPES
 INT,BIGINT,DECIMAL,STRING
 ====
@@ -255,6 +261,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/multi_col_truncate/data/s_trunc
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/multi_col_truncate/data/s_trunc=the
 quick brown/i_trunc=5/b_trunc=330/d_trunc=113211.200000/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/multi_col_truncate/data/s_trunc=the
 quick fox b/i_trunc=5/b_trunc=440/d_trunc=1111154.100000/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/multi_col_truncate/data/s_trunc=the
 quick impal/i_trunc=15/b_trunc=550/d_trunc=9999913.200000/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/multi_col_truncate/data/s_trunc=__HIVE_DEFAULT_PARTITION__/i_trunc=__HIVE_DEFAULT_PARTITION__/b_trunc=__HIVE_DEFAULT_PARTITION__/d_trunc=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -293,7 +300,7 @@ where i < 5;
 INT,BIGINT,DECIMAL,STRING
 ---- RUNTIME_PROFILE
 aggregation(SUM, RowsRead): 2
-aggregation(SUM, NumRowGroups): 2
+aggregation(SUM, NumRowGroups): 3
 ====
 ---- QUERY
 # Test partition pruning with RUNTIME_PROFILE.
@@ -335,6 +342,15 @@ aggregation(SUM, RowsRead): 1
 aggregation(SUM, NumRowGroups): 1
 ====
 ---- QUERY
+# Test partition pruning with RUNTIME_PROFILE.
+select * from multi_col_truncate
+where b is NULL;
+---- RESULTS
+NULL,NULL,NULL,'NULL'
+---- TYPES
+INT,BIGINT,DECIMAL,STRING
+====
+---- QUERY
 # Create table with YEAR partition transform
 create table year_transform(t timestamp, d date)
 partitioned by spec (year(t), year(d))
@@ -347,7 +363,8 @@ insert into year_transform values
 ('1970-01-01 00:00:00', '1970-01-01'),
 ('1970-11-01 00:00:00', '1970-11-01'),
 ('1969-12-15 13:55:03', '1969-12-15'),
-('1969-02-15 13:55:03', '1969-02-15');
+('1969-02-15 13:55:03', '1969-02-15'),
+(NULL, NULL);
 select * from year_transform;
 ---- RESULTS
 2021-01-08 14:04:03,2021-01-08
@@ -356,6 +373,7 @@ select * from year_transform;
 1970-11-01 00:00:00,1970-11-01
 1969-12-15 13:55:03,1969-12-15
 1969-02-15 13:55:03,1969-02-15
+NULL,NULL
 ---- TYPES
 TIMESTAMP,DATE
 ====
@@ -365,6 +383,7 @@ show files in year_transform;
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/year_transform/data/t_year=1969/d_year=1969/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/year_transform/data/t_year=1970/d_year=1970/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/year_transform/data/t_year=2021/d_year=2021/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/year_transform/data/t_year=__HIVE_DEFAULT_PARTITION__/d_year=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -416,7 +435,7 @@ where t <= '1970-01-01 00:00:00';
 TIMESTAMP,DATE
 ---- RUNTIME_PROFILE
 aggregation(SUM, RowsRead): 4
-aggregation(SUM, NumRowGroups): 2
+aggregation(SUM, NumRowGroups): 3
 ====
 ---- QUERY
 # Test partition pruning with RUNTIME_PROFILE.
@@ -467,7 +486,8 @@ insert into month_transform values
 ('1970-01-01 00:00:00', '1970-01-01'),
 ('1970-11-01 00:00:00', '1970-11-01'),
 ('1969-12-15 13:55:03', '1969-12-15'),
-('1969-02-15 13:55:03', '1969-02-15');
+('1969-02-15 13:55:03', '1969-02-15'),
+(NULL, NULL);
 select * from month_transform;
 ---- RESULTS
 2021-01-08 14:04:03,2021-01-08
@@ -476,6 +496,7 @@ select * from month_transform;
 1970-11-01 00:00:00,1970-11-01
 1969-12-15 13:55:03,1969-12-15
 1969-02-15 13:55:03,1969-02-15
+NULL,NULL
 ---- TYPES
 TIMESTAMP,DATE
 ====
@@ -487,6 +508,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/month_transform/data/t_month=19
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/month_transform/data/t_month=1970-01/d_month=1970-01/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/month_transform/data/t_month=1970-11/d_month=1970-11/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/month_transform/data/t_month=2021-01/d_month=2021-01/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/month_transform/data/t_month=__HIVE_DEFAULT_PARTITION__/d_month=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -576,7 +598,8 @@ insert into day_transform values
 ('1970-01-01 00:00:00', '1970-01-01'),
 ('1970-11-01 00:00:00', '1970-11-01'),
 ('1969-12-15 13:55:03', '1969-12-15'),
-('1969-02-15 13:55:03', '1969-02-15');
+('1969-02-15 13:55:03', '1969-02-15'),
+(NULL, NULL);
 select * from day_transform;
 ---- RESULTS
 2021-01-08 14:04:03,2021-01-08
@@ -586,6 +609,7 @@ select * from day_transform;
 1970-11-01 00:00:00,1970-11-01
 1969-12-15 13:55:03,1969-12-15
 1969-02-15 13:55:03,1969-02-15
+NULL,NULL
 ---- TYPES
 TIMESTAMP,DATE
 ====
@@ -599,6 +623,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/day_transform/data/t_day=1970-1
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/day_transform/data/t_day=2021-01-07/d_day=2021-01-07/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/day_transform/data/t_day=2021-01-08/d_day=2021-01-08/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/day_transform/data/t_day=2021-01-08/d_day=2021-01-08/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/day_transform/data/t_day=__HIVE_DEFAULT_PARTITION__/d_day=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -702,7 +727,8 @@ insert into hour_transform values
 ('1970-01-01 01:00:00'),
 ('2021-01-08 14:04:03'),
 ('2021-01-08 15:04:03'),
-('2021-01-08 15:04:33');
+('2021-01-08 15:04:33'),
+(NULL);
 select * from hour_transform;
 ---- RESULTS
 1969-12-31 22:55:03
@@ -714,6 +740,7 @@ select * from hour_transform;
 2021-01-08 14:04:03
 2021-01-08 15:04:03
 2021-01-08 15:04:33
+NULL
 ---- TYPES
 TIMESTAMP
 ====
@@ -726,6 +753,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/hour_transform/data/t_hour=1970
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/hour_transform/data/t_hour=1970-01-01-01/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/hour_transform/data/t_hour=2021-01-08-14/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/hour_transform/data/t_hour=2021-01-08-15/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/hour_transform/data/t_hour=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -767,7 +795,7 @@ where t < '1970-01-01 01:00:00';
 TIMESTAMP
 ---- RUNTIME_PROFILE
 aggregation(SUM, RowsRead): 5
-aggregation(SUM, NumRowGroups): 3
+aggregation(SUM, NumRowGroups): 4
 ====
 ---- QUERY
 # Test partition pruning with RUNTIME_PROFILE.
@@ -795,7 +823,8 @@ insert into mixed_and_shuffled values
 ('quick brown impala', 54321, 543.21, '2021-01-01 02:03:04', '2021-01-01'),
 ('quick brown dog', 31, 3333.45, '2020-01-06 05:06:07', '2020-01-06'),
 ('pink parrot', 9999777999, 9.9, '1999-09-09 09:09:09', '1999-09-09'),
-('green python', 717171, 71, '1971-07-01 07:07:07', '1971-07-01');
+('green python', 717171, 71, '1971-07-01 07:07:07', '1971-07-01'),
+(NULL, NULL, NULL, NULL, NULL);
 select * from mixed_and_shuffled;
 ---- RESULTS
 'quick brown fox',12345,123.45,2021-01-01 01:02:03,2021-01-01
@@ -803,6 +832,7 @@ select * from mixed_and_shuffled;
 'quick brown dog',31,3333.45,2020-01-06 05:06:07,2020-01-06
 'pink parrot',9999777999,9.90,1999-09-09 09:09:09,1999-09-09
 'green python',717171,71.00,1971-07-01 07:07:07,1971-07-01
+'NULL',NULL,NULL,NULL,NULL
 ---- TYPES
 STRING,BIGINT,DECIMAL,TIMESTAMP,DATE
 ====
@@ -814,6 +844,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/mixed_and_shuffled/data/t_day=1
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/mixed_and_shuffled/data/t_day=2020-01-06/da_year=2020/s_trunc=quick/b_bucket=1/de_trunc=3333.00/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/mixed_and_shuffled/data/t_day=2021-01-01/da_year=2021/s_trunc=quick/b_bucket=1/de_trunc=543.00/.*.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/mixed_and_shuffled/data/t_day=2021-01-01/da_year=2021/s_trunc=quick/b_bucket=2/de_trunc=123.00/.*.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/mixed_and_shuffled/data/t_day=__HIVE_DEFAULT_PARTITION__/da_year=__HIVE_DEFAULT_PARTITION__/s_trunc=__HIVE_DEFAULT_PARTITION__/b_bucket=__HIVE_DEFAULT_PARTITION__/de_trunc=__HIVE_DEFAULT_PARTITION__/.*.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -867,3 +898,23 @@ STRING,BIGINT,DECIMAL,TIMESTAMP,DATE
 aggregation(SUM, RowsRead): 3
 aggregation(SUM, NumRowGroups): 3
 ====
+---- QUERY
+select * from mixed_and_shuffled
+where s is null;
+---- RESULTS
+'NULL',NULL,NULL,NULL,NULL
+---- TYPES
+STRING,BIGINT,DECIMAL,TIMESTAMP,DATE
+====
+---- QUERY
+select * from mixed_and_shuffled
+where da is not null;
+---- RESULTS
+'quick brown fox',12345,123.45,2021-01-01 01:02:03,2021-01-01
+'quick brown impala',54321,543.21,2021-01-01 02:03:04,2021-01-01
+'quick brown dog',31,3333.45,2020-01-06 05:06:07,2020-01-06
+'pink parrot',9999777999,9.90,1999-09-09 09:09:09,1999-09-09
+'green python',717171,71.00,1971-07-01 07:07:07,1971-07-01
+---- TYPES
+STRING,BIGINT,DECIMAL,TIMESTAMP,DATE
+====
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
index 090edff..36c329e 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
@@ -11,12 +11,14 @@ stored as iceberg;
 ---- QUERY
 insert into ice_only_part values (1), (2), (3);
 insert into ice_only_part values (cast(4 as tinyint));
+insert into ice_only_part values (NULL);
 select * from ice_only_part;
 ---- RESULTS
 1
 2
 3
 4
+NULL
 ---- TYPES
 INT
 ====
@@ -27,6 +29,7 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_only_part/data/i=1/.*.0.par
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_only_part/data/i=2/.*.0.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_only_part/data/i=3/.*.0.parq','.*',''
 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_only_part/data/i=4/.*.0.parq','.*',''
+row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_only_part/data/i=__HIVE_DEFAULT_PARTITION__/.*.0.parq','.*',''
 ---- TYPES
 STRING, STRING, STRING
 ====
@@ -419,3 +422,19 @@ 
row_regex:'$NAMENODE/test-warehouse/$DATABASE.db/ice_void/data/[^=/]*.0.parq','.
 ---- TYPES
 STRING, STRING, STRING
 ====
+---- QUERY
+create table store_sales partitioned by spec (ss_sold_date_sk) stored as 
iceberg
+as select * from tpcds_parquet.store_sales;
+select count(*) from store_sales;
+---- RESULTS
+2880404
+---- TYPES
+BIGINT
+====
+---- QUERY
+select count(*) from store_sales where ss_sold_date_sk is null;
+---- RESULTS
+130093
+---- TYPES
+BIGINT
+====

Reply via email to