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

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

commit 637c750a3e83e8cf01ee0bd5141087d423cf6253
Author: zhangyifan27 <[email protected]>
AuthorDate: Fri Dec 15 18:19:58 2023 +0800

    IMPALA-12631: Improve count star performance for parquet scans
    
    Before this patch frontend generates multiple scan ranges for a
    parquet file when count star optimization is enabled. Backend function
    HdfsParquetScanner::GetNextInternal() also call NextRowGroup()
    multiple times to find row groups and sum up RowGroup.num_rows. This
    could be inefficient because we only need to read file metadata to
    compute count star. This patch optimizes it by creating only one
    scan range that contains the file footer for each parquet file.
    
    The following table shows a performance comparison before and after
    the patch. primitive_count_star_multiblock query is a modified
    primitive_count_star query that targets a multi-block
    tpch10_parquet.lineitem table. The files of the table are generated
    by the command `hdfs dfs -Ddfs.block.size=1048576 -cp -f -d`.
    
    
+-------------------+---------------------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
    | Workload          | Query                           | File Format         
  | Avg(s) | Base Avg(s) | Delta(Avg) | StdDev(%)  | Base StdDev(%) | Iters | 
Median Diff(%) | MW Zval | Tval   |
    
+-------------------+---------------------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
    | TPCDS(10)         | TPCDS-Q_COUNT_OPTIMIZED         | parquet / none / 
none | 0.17   | 0.16        |   +2.58%   | * 29.53% * | * 27.16% *     | 30    
|   +1.20%       | 0.58    | 0.35   |
    | TPCDS(10)         | TPCDS-Q_COUNT_UNOPTIMIZED       | parquet / none / 
none | 0.27   | 0.26        |   +2.96%   |   8.97%    |   9.94%        | 30    
|   +0.16%       | 0.44    | 1.19   |
    | TPCDS(10)         | TPCDS-Q_COUNT_ZERO_SLOT         | parquet / none / 
none | 0.18   | 0.18        |   -0.69%   |   1.65%    |   1.99%        | 30    
|   -0.34%       | -1.55   | -1.47  |
    | TARGETED-PERF(10) | primitive_count_star_multiblock | parquet / none / 
none | 0.06   | 0.12        | I -49.88%  |   4.11%    |   3.53%        | 30    
| I -99.97%      | -6.54   | -66.81 |
    
+-------------------+---------------------------------+-----------------------+--------+-------------+------------+------------+----------------+-------+----------------+---------+--------+
    
    Testing:
    - Ran PlannerTest#testParquetStatsAgg
    - Added new test cases to query_test/test_aggregation.py
    
    Change-Id: Ib9cd2448fe51a420d4559d0cc861c4d30822f4fd
    Reviewed-on: http://gerrit.cloudera.org:8080/20804
    Reviewed-by: Zoltan Borok-Nagy <[email protected]>
    Tested-by: Impala Public Jenkins <[email protected]>
---
 be/src/exec/parquet/hdfs-parquet-scanner.cc        | 27 ++++-----
 .../org/apache/impala/planner/HdfsScanNode.java    | 69 ++++++++++++++--------
 .../queries/QueryTest/hdfs-tiny-scan.test          |  7 +++
 .../QueryTest/iceberg-in-predicate-push-down.test  |  6 +-
 .../QueryTest/iceberg-partitioned-insert.test      | 18 ++----
 .../iceberg-plain-count-star-optimization.test     |  6 +-
 .../queries/QueryTest/parquet-stats-agg.test       | 42 +++++++++++++
 .../tpcds-decimal_v2-q_count_optimized.test        |  8 +++
 .../tpcds-decimal_v2-q_count_unoptimized.test      |  8 +++
 .../tpcds-decimal_v2-q_count_zero_slot.test        |  8 +++
 tests/util/parse_util.py                           |  2 +-
 11 files changed, 138 insertions(+), 63 deletions(-)

diff --git a/be/src/exec/parquet/hdfs-parquet-scanner.cc 
b/be/src/exec/parquet/hdfs-parquet-scanner.cc
index 8ff1cc312..d69e985d4 100644
--- a/be/src/exec/parquet/hdfs-parquet-scanner.cc
+++ b/be/src/exec/parquet/hdfs-parquet-scanner.cc
@@ -437,31 +437,26 @@ Status HdfsParquetScanner::GetNextInternal(RowBatch* 
row_batch) {
     // Populate the single slot with the Parquet num rows statistic.
     int64_t tuple_buf_size;
     uint8_t* tuple_buf;
-    // We try to allocate a smaller row batch here because in most cases the 
number row
-    // groups in a file is much lower than the default row batch capacity.
-    int capacity = min(
-        static_cast<int>(file_metadata_.row_groups.size()), 
row_batch->capacity());
-    RETURN_IF_ERROR(RowBatch::ResizeAndAllocateTupleBuffer(state_,
-        row_batch->tuple_data_pool(), row_batch->row_desc()->GetRowSize(),
-        &capacity, &tuple_buf_size, &tuple_buf));
-    while (!row_batch->AtCapacity()) {
-      RETURN_IF_ERROR(NextRowGroup());
-      DCHECK_LE(row_group_idx_, file_metadata_.row_groups.size());
-      DCHECK_LE(row_group_rows_read_, file_metadata_.num_rows);
-      if (row_group_idx_ == file_metadata_.row_groups.size()) break;
+    int capacity = 1;
+    RETURN_IF_ERROR(
+        RowBatch::ResizeAndAllocateTupleBuffer(state_, 
row_batch->tuple_data_pool(),
+            row_batch->row_desc()->GetRowSize(), &capacity, &tuple_buf_size, 
&tuple_buf));
+    if (file_metadata_.num_rows > 0) {
       COUNTER_ADD(num_file_metadata_read_, 1);
       Tuple* dst_tuple = reinterpret_cast<Tuple*>(tuple_buf);
       TupleRow* dst_row = row_batch->GetRow(row_batch->AddRow());
       InitTuple(template_tuple_, dst_tuple);
       int64_t* dst_slot =
           
dst_tuple->GetBigIntSlot(scan_node_->parquet_count_star_slot_offset());
-      *dst_slot = file_metadata_.row_groups[row_group_idx_].num_rows;
-      row_group_rows_read_ += *dst_slot;
+      *dst_slot = 0;
+      for (const auto &row_group : file_metadata_.row_groups) {
+        *dst_slot += row_group.num_rows;
+      }
       dst_row->SetTuple(0, dst_tuple);
       row_batch->CommitLastRow();
-      tuple_buf += scan_node_->tuple_desc()->byte_size();
+      row_group_rows_read_ += *dst_slot;
     }
-    eos_ = row_group_idx_ == file_metadata_.row_groups.size();
+    eos_ = true;
     return Status::OK();
   } else if (scan_node_->IsZeroSlotTableScan()) {
     // There are no materialized slots and we are not optimizing count(*), e.g.
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java 
b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index a6e84d222..ed926adef 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -447,12 +447,6 @@ public class HdfsScanNode extends ScanNode {
       }
     }
 
-    if (canApplyCountStarOptimization(analyzer, fileFormats_)) {
-      Preconditions.checkState(desc_.getPath().destTable() != null);
-      Preconditions.checkState(collectionConjuncts_.isEmpty());
-      countStarSlot_ = applyCountStarOptimization(analyzer);
-    }
-
     computeMemLayout(analyzer);
 
     // This is towards the end, so that it can take all conjuncts, scan ranges 
and mem
@@ -1121,6 +1115,26 @@ public class HdfsScanNode extends ScanNode {
       sampledFiles_ = getFilesSample(percentBytes, 0, randomSeed);
     }
 
+    // Get partitions in the sample and initialize fileFormats_.
+    List<FeFsPartition> partitions = new ArrayList<>();
+    for (FeFsPartition partition : partitions_) {
+      Preconditions.checkState(partition.getId() >= 0);
+      if (sampledFiles_ != null && sampledFiles_.get(partition.getId()) == 
null) {
+        // If we are sampling, check whether this partition is included in the 
sample.
+        continue;
+      }
+      partitions.add(partition);
+      if (partition.getFileFormat() != HdfsFileFormat.ICEBERG) {
+        fileFormats_.add(partition.getFileFormat());
+      }
+    }
+
+    if (canApplyCountStarOptimization(analyzer, fileFormats_)) {
+      Preconditions.checkState(desc_.getPath().destTable() != null);
+      Preconditions.checkState(collectionConjuncts_.isEmpty());
+      countStarSlot_ = applyCountStarOptimization(analyzer);
+    }
+
     long scanRangeBytesLimit = 
analyzer.getQueryCtx().client_request.getQuery_options()
         .getMax_scan_range_length();
     if (RuntimeEnv.INSTANCE.hasTableScanRangeLimit() && desc_.getTableName() 
!= null) {
@@ -1158,7 +1172,7 @@ public class HdfsScanNode extends ScanNode {
     String lastFsScheme = null;
     String lastFsAuthority = null;
     FileSystem lastFileSytem = null;
-    for (FeFsPartition partition: partitions_) {
+    for (FeFsPartition partition : partitions) {
       // Save location to local variable beacuse getLocation() can be slow as 
it needs to
       // decompress the partition's location.
       String partitionLocation = partition.getLocation();
@@ -1191,25 +1205,18 @@ public class HdfsScanNode extends ScanNode {
                         : -1);
       // conservatively estimate 1 row per file
       simpleLimitNumRows += fileDescs.size();
-
       if (sampledFiles_ != null) {
-        // If we are sampling, check whether this partition is included in the 
sample.
+        // If we are sampling, get files in the sample.
         fileDescs = sampledFiles_.get(partition.getId());
-        if (fileDescs == null) continue;
       }
 
       long partitionNumRows = partition.getNumRows();
-
       analyzer.getDescTbl().addReferencedPartition(tbl_, partition.getId());
-      if (partition.getFileFormat() != HdfsFileFormat.ICEBERG) {
-        fileFormats_.add(partition.getFileFormat());
-      }
       if (!partition.getFileFormat().isParquetBased()) {
         allParquet = false;
       }
       allColumnarFormat =
           allColumnarFormat && 
VALID_COLUMNAR_FORMATS.contains(partition.getFileFormat());
-      Preconditions.checkState(partition.getId() >= 0);
 
       if (!fsHasBlocks) {
         // Limit the scan range length if generating scan ranges (and we're not
@@ -1253,9 +1260,19 @@ public class HdfsScanNode extends ScanNode {
         } else {
           // Skips files that have no associated blocks.
           if (fileDesc.getNumFileBlocks() == 0) continue;
-          Pair<Boolean, Long> result = transformBlocksToScanRanges(
-              partition, partitionLocation, fsType, fileDesc, fsHasBlocks,
-              scanRangeBytesLimit, analyzer);
+          // If parquet count star optimization is enabled, we only need the
+          // 'RowGroup.num_rows' in file metadata, thus only the scan range 
that contains
+          // a file footer is required.
+          // IMPALA-8834 introduced the optimization for partition key scan by 
generating
+          // one scan range for each HDFS file. With Parquet and ORC, we only 
need to get
+          // the scan range that contains a file footer for short-circuiting.
+          boolean isFooterOnly = countStarSlot_ != null
+              || (isPartitionKeyScan_
+                  && (partition.getFileFormat().isParquetBased()
+                      || partition.getFileFormat() == HdfsFileFormat.ORC));
+          Pair<Boolean, Long> result =
+              transformBlocksToScanRanges(partition, partitionLocation, 
fsType, fileDesc,
+                  fsHasBlocks, scanRangeBytesLimit, analyzer, isFooterOnly);
           partitionMaxScanRangeBytes =
               Math.max(partitionMaxScanRangeBytes, result.second);
           if (result.first) partitionMissingDiskIds = true;
@@ -1388,18 +1405,13 @@ public class HdfsScanNode extends ScanNode {
    */
   private Pair<Boolean, Long> transformBlocksToScanRanges(FeFsPartition 
partition,
       String partitionLocation, FileSystemUtil.FsType fsType, FileDescriptor 
fileDesc,
-      boolean fsHasBlocks, long scanRangeBytesLimit, Analyzer analyzer) {
+      boolean fsHasBlocks, long scanRangeBytesLimit, Analyzer analyzer,
+      boolean isFooterOnly) {
     Preconditions.checkArgument(fileDesc.getNumFileBlocks() > 0);
     boolean fileDescMissingDiskIds = false;
     long fileMaxScanRangeBytes = 0;
     int i = 0;
-    if (isPartitionKeyScan_ && (partition.getFileFormat().isParquetBased()
-        || partition.getFileFormat() == HdfsFileFormat.ORC)) {
-      // IMPALA-8834 introduced the optimization for partition key scan by 
generating
-      // one scan range for each HDFS file. With Parquet and ORC, we start 
with the last
-      // block to get a scan range that contains a file footer for 
short-circuiting.
-      i = fileDesc.getNumFileBlocks() - 1;
-    }
+    if (isFooterOnly) { i = fileDesc.getNumFileBlocks() - 1; }
     for (; i < fileDesc.getNumFileBlocks(); ++i) {
       FbFileBlock block = fileDesc.getFbFileBlock(i);
       int replicaHostCount = FileBlock.getNumReplicaHosts(block);
@@ -1438,6 +1450,11 @@ public class HdfsScanNode extends ScanNode {
         long currentLength = remainingLength;
         if (scanRangeBytesLimit > 0 && remainingLength > scanRangeBytesLimit) {
           currentLength = scanRangeBytesLimit;
+          if (isFooterOnly) {
+            // Only generate one scan range for footer only scans.
+            currentOffset += remainingLength - currentLength;
+            remainingLength = currentLength;
+          }
         }
         TScanRange scanRange = new TScanRange();
         THdfsFileSplit hdfsFileSplit = new 
THdfsFileSplit(fileDesc.getRelativePath(),
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/hdfs-tiny-scan.test 
b/testdata/workloads/functional-query/queries/QueryTest/hdfs-tiny-scan.test
index a6301ff5c..26483ec35 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/hdfs-tiny-scan.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/hdfs-tiny-scan.test
@@ -24,3 +24,10 @@ bigint
 ---- RESULTS
 100
 ====
+---- QUERY
+select max(year) from alltypessmall
+---- TYPES
+int
+---- RESULTS
+2009
+====
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-in-predicate-push-down.test
 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-in-predicate-push-down.test
index 866bc582f..fcec4cb69 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-in-predicate-push-down.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-in-predicate-push-down.test
@@ -63,7 +63,6 @@ aggregation(SUM, NumRowGroups): 3
 ---- QUERY
 # Filtering only on a partition column is done by Iceberg and in Impala we can 
get the results
 # simply using file metadata.
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 3.
 select
   count(1)
 from
@@ -73,7 +72,7 @@ where
 ---- RESULTS
 9
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 3
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 3
 ====
 ---- QUERY
@@ -448,7 +447,6 @@ aggregation(SUM, NumRowGroups): 1
 ====
 ---- QUERY
 # NOT_IN could be answered using file metadata if only partition cols are 
included
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 1.
 select
   count(1)
 from
@@ -458,7 +456,7 @@ where
 ---- RESULTS
 3
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 1
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 1
 ====
 ---- QUERY
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 e54a8f6af..da70926fa 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-partitioned-insert.test
@@ -174,13 +174,12 @@ aggregation(SUM, NumFileMetadataRead): 0
 ====
 ---- QUERY
 # When filtered only by partition column Iceberg can do the filtering and no 
need to read data in Impala.
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 1.
 select count(*) from ice_bigints
 where i = 0 and j = 0;
 ---- RESULTS
 1217
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 1
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, RowsRead): 0
 aggregation(SUM, NumFileMetadataRead): 1
 ====
@@ -282,7 +281,6 @@ select count(*) from alltypes_part;
 Expression 'timestamp_col' (type: TIMESTAMP) would need to be cast to STRING 
for column 'string_col'
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 4.
 select count(*) from alltypes_part
 where bool_col = true;
 ---- RESULTS
@@ -290,11 +288,10 @@ where bool_col = true;
 ---- TYPES
 BIGINT
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 4
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 4
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 4.
 select count(*) from alltypes_part
 where float_col = 0;
 ---- RESULTS
@@ -302,11 +299,10 @@ where float_col = 0;
 ---- TYPES
 BIGINT
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 4
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 4
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 4.
 select count(*) from alltypes_part
 where double_col = 0;
 ---- RESULTS
@@ -314,11 +310,10 @@ where double_col = 0;
 ---- TYPES
 BIGINT
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 4
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 4
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 2.
 select count(*) from alltypes_part
 where date_col = '2009-01-01';
 ---- RESULTS
@@ -326,11 +321,10 @@ where date_col = '2009-01-01';
 ---- TYPES
 BIGINT
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 2
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 2
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 4.
 select count(*) from alltypes_part
 where string_col = '0';
 ---- RESULTS
@@ -338,7 +332,7 @@ where string_col = '0';
 ---- TYPES
 BIGINT
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 4
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 4
 ====
 ---- QUERY
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-plain-count-star-optimization.test
 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-plain-count-star-optimization.test
index f43346d36..da56563eb 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/iceberg-plain-count-star-optimization.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/iceberg-plain-count-star-optimization.test
@@ -145,7 +145,6 @@ explain select 123, count(*), 321 from ice_tbl;
 ====
 ---- QUERY
 # Filtering by a partition column results in Iceberg doing the filtering 
instead of Impala.
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 2.
 select
   count(*)
 from
@@ -155,7 +154,7 @@ where
 ---- RESULTS
 4
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 2
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 2
 ====
 ---- QUERY
@@ -214,7 +213,6 @@ create table parq_tbl(col_i INT, col_s STRING) PARTITIONED 
BY(x INT) STORED AS P
 'Table has been created.'
 ====
 ---- QUERY
-# IMPALA-11123: Behavior changes after a revert: NumRowGroups changed from 0 
to 3.
 insert into parq_tbl PARTITION(x = 12340) values (0, "a");
 insert into parq_tbl PARTITION(x = 12341) values (1, "b");
 insert into parq_tbl PARTITION(x = 12342) values (2, "c");
@@ -222,7 +220,7 @@ select count(*) from parq_tbl;
 ---- RESULTS
 3
 ---- RUNTIME_PROFILE
-aggregation(SUM, NumRowGroups): 3
+aggregation(SUM, NumRowGroups): 0
 aggregation(SUM, NumFileMetadataRead): 3
 ====
 ---- QUERY
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/parquet-stats-agg.test 
b/testdata/workloads/functional-query/queries/QueryTest/parquet-stats-agg.test
index 620c50bef..3b18de839 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/parquet-stats-agg.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/parquet-stats-agg.test
@@ -7,6 +7,10 @@ from functional_parquet.alltypes
 7300
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 24
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Parquet count(*) optimization with predicates on the partition columns.
@@ -16,6 +20,10 @@ from functional_parquet.alltypes where year < 2010 and month 
> 8
 1220
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 4
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Parquet count(*) optimization with group by partition columns.
@@ -48,6 +56,10 @@ from functional_parquet.alltypes group by year, month
 2010,12,310
 ---- TYPES
 int, int, bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 24
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Parquet count(*) optimization with both group by and predicates on partition 
columns.
@@ -61,6 +73,10 @@ group by month
 300
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 4
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Parquet count(*) optimization with the result going into a join.
@@ -73,6 +89,10 @@ select x.bigint_col from functional.alltypes x
 0
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 24
+aggregation(SUM, RowsRead): 7300
 =====
 ---- QUERY
 # Parquet count(*) optimization with the agg function in the having clause.
@@ -81,6 +101,10 @@ select 1 from functional_parquet.alltypes having count(*) > 
1
 1
 ---- TYPES
 tinyint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 24
+aggregation(SUM, RowsRead): 0
 ====
 ---- QUERY
 # Verify that 0 is returned for count(*) on an empty table.
@@ -89,6 +113,10 @@ select count(1) from functional_parquet.emptytable
 0
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 0
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Verify that 0 is returned when all partitions are pruned.
@@ -97,11 +125,17 @@ select count(1) from functional_parquet.alltypes where 
year = -1
 0
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 0
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # Test different row group size combinations.
 select count(*) from functional_parquet.lineitem_multiblock
 union all
+select count(*) from functional_parquet.lineitem_multiblock_variable_num_rows
+union all
 select count(*) from functional_parquet.lineitem_multiblock_one_row_group
 union all
 select count(*) from functional_parquet.lineitem_sixblocks
@@ -109,11 +143,15 @@ union all
 select count(*) from tpch_parquet.lineitem
 ---- RESULTS
 20000
+20000
 40000
 40000
 6001215
 ---- TYPES
 bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, RowsRead): 0
 =====
 ---- QUERY
 # IMPALA-5679: Count(*) with group by on a string partition column.
@@ -136,4 +174,8 @@ select string_col, count(*) from 
$DATABASE.string_partitioned_table group by str
 '9',730
 ---- TYPES
 string, bigint
+---- RUNTIME_PROFILE
+aggregation(SUM, NumRowGroups): 0
+aggregation(SUM, NumFileMetadataRead): 10
+aggregation(SUM, RowsRead): 0
 =====
diff --git 
a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_optimized.test 
b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_optimized.test
new file mode 100644
index 000000000..e6f8921c6
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_optimized.test
@@ -0,0 +1,8 @@
+====
+---- QUERY: TPCDS-Q_COUNT_OPTIMIZED
+select count(*) from store_sales;
+---- RESULTS
+2880404
+---- TYPES
+BIGINT
+====
diff --git 
a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_unoptimized.test 
b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_unoptimized.test
new file mode 100644
index 000000000..c23dc5b2c
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_unoptimized.test
@@ -0,0 +1,8 @@
+====
+---- QUERY: TPCDS-Q_COUNT_UNOPTIMIZED
+select count(*) from store_sales where ss_ext_discount_amt != 0.3857
+---- RESULTS
+2750566
+---- TYPES
+BIGINT
+====
diff --git 
a/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_zero_slot.test 
b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_zero_slot.test
new file mode 100644
index 000000000..c7f3742e0
--- /dev/null
+++ b/testdata/workloads/tpcds/queries/tpcds-decimal_v2-q_count_zero_slot.test
@@ -0,0 +1,8 @@
+====
+---- QUERY: TPCDS-Q_COUNT_ZERO_SLOT
+select count(ss_sold_date_sk) from store_sales;
+---- RESULTS
+2750311
+---- TYPES
+BIGINT
+====
diff --git a/tests/util/parse_util.py b/tests/util/parse_util.py
index bceb6ddf8..79911c796 100644
--- a/tests/util/parse_util.py
+++ b/tests/util/parse_util.py
@@ -23,7 +23,7 @@ from datetime import datetime
 # changed, and the stress test loses the ability to run the full set of 
queries. Set
 # these constants and assert that when a workload is used, all the queries we 
expect to
 # use are there.
-EXPECTED_TPCDS_QUERIES_COUNT = 111
+EXPECTED_TPCDS_QUERIES_COUNT = 114
 EXPECTED_TPCH_NESTED_QUERIES_COUNT = 22
 EXPECTED_TPCH_QUERIES_COUNT = 22
 # Add the number of stress test specific queries, i.e. in files like 
'*-stress-*.test'

Reply via email to