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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9d0110e715d [enhance](variant) Add some metrics for querying variant 
in the profile (#55941)
9d0110e715d is described below

commit 9d0110e715d877d7ec072de19e116152dfb565a3
Author: Sun Chenyang <[email protected]>
AuthorDate: Thu Sep 18 22:23:55 2025 +0800

    [enhance](variant) Add some metrics for querying variant in the profile 
(#55941)
---
 be/src/exec/rowid_fetcher.cpp                      | 17 +++++++++++-----
 be/src/olap/olap_common.h                          |  8 ++++++++
 be/src/olap/rowset/segment_v2/segment.cpp          | 19 ++++++++----------
 be/src/olap/rowset/segment_v2/segment.h            |  3 ++-
 .../variant/hierarchical_data_iterator.cpp         |  8 ++++++--
 .../variant/hierarchical_data_iterator.h           |  9 +++++++--
 .../variant/sparse_column_extract_iterator.h       | 23 ++++++++++++++--------
 .../segment_v2/variant/variant_column_reader.cpp   |  7 ++++++-
 be/src/pipeline/exec/olap_scan_operator.cpp        | 13 ++++++++++++
 be/src/pipeline/exec/olap_scan_operator.h          | 16 +++++++++++++++
 be/src/service/point_query_executor.cpp            |  6 +++++-
 be/src/vec/exec/scan/olap_scanner.cpp              | 14 +++++++++++++
 .../variant_column_writer_reader_test.cpp          | 23 ++++++++++++++++------
 13 files changed, 129 insertions(+), 37 deletions(-)

diff --git a/be/src/exec/rowid_fetcher.cpp b/be/src/exec/rowid_fetcher.cpp
index 83cc869d6a0..00e3c15e7ba 100644
--- a/be/src/exec/rowid_fetcher.cpp
+++ b/be/src/exec/rowid_fetcher.cpp
@@ -333,6 +333,8 @@ struct IteratorItem {
     std::unique_ptr<ColumnIterator> iterator;
     // for holding the reference of segment to avoid use after release
     SegmentSharedPtr segment;
+    // for holding the reference of storage read options to avoid use after 
release
+    StorageReadOptions storage_read_options;
 };
 
 Status RowIdStorageReader::read_by_rowids(const PMultiGetRequest& request,
@@ -446,10 +448,13 @@ Status RowIdStorageReader::read_by_rowids(const 
PMultiGetRequest& request,
             if (iterator_item.segment == nullptr) {
                 // hold the reference
                 iterator_map[iterator_key].segment = segment;
+                iterator_item.storage_read_options.stats = &stats;
+                iterator_item.storage_read_options.io_ctx.reader_type = 
ReaderType::READER_QUERY;
             }
             segment = iterator_item.segment;
-            RETURN_IF_ERROR(segment->seek_and_read_by_rowid(full_read_schema, 
&slots[x], row_id,
-                                                            column, stats, 
iterator_item.iterator));
+            RETURN_IF_ERROR(segment->seek_and_read_by_rowid(
+                    full_read_schema, &slots[x], row_id, column, 
iterator_item.storage_read_options,
+                    iterator_item.iterator));
         }
     }
     // serialize block if not empty
@@ -1045,11 +1050,13 @@ Status RowIdStorageReader::read_doris_format_row(
             IteratorItem& iterator_item = iterator_map[iterator_key];
             if (iterator_item.segment == nullptr) {
                 iterator_map[iterator_key].segment = segment;
+                iterator_item.storage_read_options.stats = &stats;
+                iterator_item.storage_read_options.io_ctx.reader_type = 
ReaderType::READER_QUERY;
             }
             segment = iterator_item.segment;
-            RETURN_IF_ERROR(segment->seek_and_read_by_rowid(full_read_schema, 
&slots[x],
-                                                            
cast_set<uint32_t>(row_id), column,
-                                                            stats, 
iterator_item.iterator));
+            RETURN_IF_ERROR(segment->seek_and_read_by_rowid(
+                    full_read_schema, &slots[x], cast_set<uint32_t>(row_id), 
column,
+                    iterator_item.storage_read_options, 
iterator_item.iterator));
         }
     }
 
diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h
index 20d8df3516e..2c9425385cf 100644
--- a/be/src/olap/olap_common.h
+++ b/be/src/olap/olap_common.h
@@ -440,6 +440,14 @@ struct OlapReaderStatistics {
 
     int64_t segment_create_column_readers_timer_ns = 0;
     int64_t segment_load_index_timer_ns = 0;
+
+    int64_t variant_scan_sparse_column_timer_ns = 0;
+    int64_t variant_scan_sparse_column_bytes = 0;
+    int64_t variant_fill_path_from_sparse_column_timer_ns = 0;
+    int64_t variant_subtree_default_iter_count = 0;
+    int64_t variant_subtree_leaf_iter_count = 0;
+    int64_t variant_subtree_hierarchical_iter_count = 0;
+    int64_t variant_subtree_sparse_iter_count = 0;
 };
 
 using ColumnId = uint32_t;
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp 
b/be/src/olap/rowset/segment_v2/segment.cpp
index 95cb80a9abe..eadf3abf3d8 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -987,24 +987,22 @@ bool Segment::same_with_storage_type(int32_t cid, const 
Schema& schema, bool rea
 
 Status Segment::seek_and_read_by_rowid(const TabletSchema& schema, 
SlotDescriptor* slot,
                                        uint32_t row_id, 
vectorized::MutableColumnPtr& result,
-                                       OlapReaderStatistics& stats,
+                                       StorageReadOptions& 
storage_read_options,
                                        std::unique_ptr<ColumnIterator>& 
iterator_hint) {
-    StorageReadOptions storage_read_opt;
-    storage_read_opt.stats = &stats;
-    storage_read_opt.io_ctx.reader_type = ReaderType::READER_QUERY;
     segment_v2::ColumnIteratorOptions opt {
             .use_page_cache = !config::disable_storage_page_cache,
             .file_reader = file_reader().get(),
-            .stats = &stats,
+            .stats = storage_read_options.stats,
             .io_ctx = io::IOContext {.reader_type = ReaderType::READER_QUERY,
-                                     .file_cache_stats = 
&stats.file_cache_stats},
+                                     .file_cache_stats =
+                                             
&storage_read_options.stats->file_cache_stats},
     };
 
     std::vector<segment_v2::rowid_t> single_row_loc {row_id};
     if (!slot->column_paths().empty()) {
         // here need create column readers to make sure column reader is 
created before seek_and_read_by_rowid
         // if segment cache miss, column reader will be created to make sure 
the variant column result not coredump
-        RETURN_IF_ERROR(_create_column_meta_once(&stats));
+        RETURN_IF_ERROR(_create_column_meta_once(storage_read_options.stats));
 
         TabletColumn column = TabletColumn::create_materialized_variant_column(
                 schema.column_by_uid(slot->col_unique_id()).name_lower_case(), 
slot->column_paths(),
@@ -1016,7 +1014,7 @@ Status Segment::seek_and_read_by_rowid(const 
TabletSchema& schema, SlotDescripto
         DCHECK(storage_type != nullptr);
 
         if (iterator_hint == nullptr) {
-            RETURN_IF_ERROR(new_column_iterator(column, &iterator_hint, 
&storage_read_opt));
+            RETURN_IF_ERROR(new_column_iterator(column, &iterator_hint, 
&storage_read_options));
             RETURN_IF_ERROR(iterator_hint->init(opt));
         }
         RETURN_IF_ERROR(
@@ -1037,10 +1035,9 @@ Status Segment::seek_and_read_by_rowid(const 
TabletSchema& schema, SlotDescripto
                << ", field_name_to_index=" << schema.get_all_field_names();
             return Status::InternalError(ss.str());
         }
-        storage_read_opt.io_ctx.reader_type = ReaderType::READER_QUERY;
         if (iterator_hint == nullptr) {
-            RETURN_IF_ERROR(
-                    new_column_iterator(schema.column(index), &iterator_hint, 
&storage_read_opt));
+            RETURN_IF_ERROR(new_column_iterator(schema.column(index), 
&iterator_hint,
+                                                &storage_read_options));
             RETURN_IF_ERROR(iterator_hint->init(opt));
         }
         RETURN_IF_ERROR(iterator_hint->read_by_rowids(single_row_loc.data(), 
1, result));
diff --git a/be/src/olap/rowset/segment_v2/segment.h 
b/be/src/olap/rowset/segment_v2/segment.h
index 3f5418b1e38..098c3c082f8 100644
--- a/be/src/olap/rowset/segment_v2/segment.h
+++ b/be/src/olap/rowset/segment_v2/segment.h
@@ -142,7 +142,8 @@ public:
     Status read_key_by_rowid(uint32_t row_id, std::string* key);
 
     Status seek_and_read_by_rowid(const TabletSchema& schema, SlotDescriptor* 
slot, uint32_t row_id,
-                                  vectorized::MutableColumnPtr& result, 
OlapReaderStatistics& stats,
+                                  vectorized::MutableColumnPtr& result,
+                                  StorageReadOptions& storage_read_options,
                                   std::unique_ptr<ColumnIterator>& 
iterator_hint);
 
     Status load_index(OlapReaderStatistics* stats);
diff --git 
a/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.cpp 
b/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.cpp
index a9654783ade..b0af87641f6 100644
--- a/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.cpp
@@ -48,7 +48,7 @@ Status HierarchicalDataIterator::create(ColumnIteratorUPtr* 
reader, int32_t col_
                                         ColumnReaderCache* column_reader_cache,
                                         OlapReaderStatistics* stats) {
     // None leave node need merge with root
-    auto stream_iter = std::make_unique<HierarchicalDataIterator>(path);
+    std::unique_ptr<HierarchicalDataIterator> stream_iter(new 
HierarchicalDataIterator(path));
     if (node != nullptr) {
         std::vector<const SubcolumnColumnMetaInfo::Node*> leaves;
         vectorized::PathsInData leaves_paths;
@@ -66,6 +66,7 @@ Status HierarchicalDataIterator::create(ColumnIteratorUPtr* 
reader, int32_t col_
     stream_iter->_root_reader = std::move(root_column_reader);
     // need read from sparse column if not null
     stream_iter->_sparse_column_reader = std::move(sparse_reader);
+    stream_iter->_stats = stats;
     *reader = std::move(stream_iter);
 
     return Status::OK();
@@ -292,8 +293,11 @@ Status 
HierarchicalDataIterator::_init_container(vectorized::MutableColumnPtr& c
     RETURN_IF_ERROR(_process_sub_columns(container_variant, 
non_nested_subcolumns));
 
     RETURN_IF_ERROR(_process_nested_columns(container_variant, 
nested_subcolumns, nrows));
+    {
+        
SCOPED_RAW_TIMER(&_stats->variant_fill_path_from_sparse_column_timer_ns);
+        RETURN_IF_ERROR(_process_sparse_column(container_variant, nrows));
+    }
 
-    RETURN_IF_ERROR(_process_sparse_column(container_variant, nrows));
     container_variant.set_num_rows(nrows);
     return Status::OK();
 }
diff --git a/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.h 
b/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.h
index 3fcb3b22402..a549a131883 100644
--- a/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.h
+++ b/be/src/olap/rowset/segment_v2/variant/hierarchical_data_iterator.h
@@ -69,8 +69,6 @@ public:
     // Currently two types of read, merge sparse columns with root columns, or 
read directly
     enum class ReadType { MERGE_ROOT, READ_DIRECT };
 
-    HierarchicalDataIterator(const vectorized::PathInData& path) : _path(path) 
{}
-
     static Status create(ColumnIteratorUPtr* reader, int32_t col_uid, 
vectorized::PathInData path,
                          const SubcolumnColumnMetaInfo::Node* target_node,
                          std::unique_ptr<SubstreamIterator>&& sparse_reader,
@@ -97,6 +95,9 @@ private:
     std::unique_ptr<SubstreamIterator> _sparse_column_reader;
     size_t _rows_read = 0;
     vectorized::PathInData _path;
+    OlapReaderStatistics* _stats = nullptr;
+
+    HierarchicalDataIterator(const vectorized::PathInData& path) : _path(path) 
{}
 
     template <typename NodeFunction>
     Status tranverse(NodeFunction&& node_func) {
@@ -156,7 +157,11 @@ private:
 
         // read sparse column
         if (_sparse_column_reader) {
+            SCOPED_RAW_TIMER(&_stats->variant_scan_sparse_column_timer_ns);
+            int64_t curr_size = _sparse_column_reader->column->byte_size();
             RETURN_IF_ERROR(read_func(*_sparse_column_reader, {}, nullptr));
+            _stats->variant_scan_sparse_column_bytes +=
+                    _sparse_column_reader->column->byte_size() - curr_size;
         }
 
         MutableColumnPtr container;
diff --git 
a/be/src/olap/rowset/segment_v2/variant/sparse_column_extract_iterator.h 
b/be/src/olap/rowset/segment_v2/variant/sparse_column_extract_iterator.h
index 7d3e6bf1e09..a58fd5ca312 100644
--- a/be/src/olap/rowset/segment_v2/variant/sparse_column_extract_iterator.h
+++ b/be/src/olap/rowset/segment_v2/variant/sparse_column_extract_iterator.h
@@ -59,7 +59,7 @@ protected:
     vectorized::MutableColumnPtr _sparse_column;
     StorageReadOptions* _read_opts; // Shared cache pointer
     std::unique_ptr<ColumnIterator> _sparse_column_reader;
-    const TabletColumn& _col;
+    int32_t _col_uid;
     // Pure virtual method for data processing when encounter existing sparse 
columns(to be implemented by subclasses)
     virtual void 
_process_data_with_existing_sparse_column(vectorized::MutableColumnPtr& dst,
                                                            size_t num_rows) = 
0;
@@ -71,7 +71,9 @@ protected:
 public:
     BaseSparseColumnProcessor(std::unique_ptr<ColumnIterator>&& reader, 
StorageReadOptions* opts,
                               const TabletColumn& col)
-            : _read_opts(opts), _sparse_column_reader(std::move(reader)), 
_col(col) {
+            : _read_opts(opts),
+              _sparse_column_reader(std::move(reader)),
+              _col_uid(col.parent_unique_id()) {
         _sparse_column = vectorized::ColumnVariant::create_sparse_column_fn();
     }
 
@@ -85,7 +87,7 @@ public:
     // So we need to cache the sparse column and reuse it.
     // The cache is only used when the compaction reader is used.
     bool has_sparse_column_cache() const {
-        return _read_opts && 
_read_opts->sparse_column_cache[_col.parent_unique_id()] &&
+        return _read_opts && _read_opts->sparse_column_cache[_col_uid] &&
                
ColumnReader::is_compaction_reader_type(_read_opts->io_ctx.reader_type);
     }
 
@@ -106,18 +108,23 @@ public:
                           vectorized::MutableColumnPtr& dst) {
         // Cache check and population logic
         if (has_sparse_column_cache()) {
-            _sparse_column =
-                    
_read_opts->sparse_column_cache[_col.parent_unique_id()]->assume_mutable();
+            _sparse_column = 
_read_opts->sparse_column_cache[_col_uid]->assume_mutable();
         } else {
             _sparse_column->clear();
-            RETURN_IF_ERROR(read_method());
+            {
+                
SCOPED_RAW_TIMER(&_read_opts->stats->variant_scan_sparse_column_timer_ns);
+                int64_t before_size = 
_read_opts->stats->uncompressed_bytes_read;
+                RETURN_IF_ERROR(read_method());
+                _read_opts->stats->variant_scan_sparse_column_bytes +=
+                        _read_opts->stats->uncompressed_bytes_read - 
before_size;
+            }
 
             // cache the sparse column
             if (_read_opts) {
-                _read_opts->sparse_column_cache[_col.parent_unique_id()] =
-                        _sparse_column->get_ptr();
+                _read_opts->sparse_column_cache[_col_uid] = 
_sparse_column->get_ptr();
             }
         }
+        
SCOPED_RAW_TIMER(&_read_opts->stats->variant_fill_path_from_sparse_column_timer_ns);
 
         const auto& offsets =
                 assert_cast<const 
vectorized::ColumnMap&>(*_sparse_column).get_offsets();
diff --git a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp 
b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
index 7adbd33de80..737ab3453fe 100644
--- a/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
+++ b/be/src/olap/rowset/segment_v2/variant/variant_column_reader.cpp
@@ -116,6 +116,7 @@ Status 
VariantColumnReader::_create_hierarchical_reader(ColumnIteratorUPtr* read
                                                         const 
SubcolumnColumnMetaInfo::Node* root,
                                                         ColumnReaderCache* 
column_reader_cache,
                                                         OlapReaderStatistics* 
stats) {
+    stats->variant_subtree_hierarchical_iter_count++;
     // Node contains column with children columns or has correspoding sparse 
columns
     // Create reader with hirachical data.
     std::unique_ptr<SubstreamIterator> sparse_iter;
@@ -378,7 +379,9 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
         DCHECK(opt);
         // Sparse column exists or reached sparse size limit, read sparse 
column
         *iterator = std::make_unique<SparseColumnExtractIterator>(
-                relative_path.get_path(), std::move(inner_iter), nullptr, 
*target_col);
+                relative_path.get_path(), std::move(inner_iter),
+                const_cast<StorageReadOptions*>(opt), *target_col);
+        opt->stats->variant_subtree_sparse_iter_count++;
         return Status::OK();
     }
 
@@ -392,6 +395,7 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
             RETURN_IF_ERROR(column_reader_cache->get_path_column_reader(
                     col_uid, leaf_node->path, &leaf_column_reader, opt->stats, 
leaf_node));
             RETURN_IF_ERROR(leaf_column_reader->new_iterator(iterator, 
nullptr));
+            opt->stats->variant_subtree_leaf_iter_count++;
         } else {
             RETURN_IF_ERROR(_create_hierarchical_reader(iterator, col_uid, 
relative_path, node,
                                                         root, 
column_reader_cache, opt->stats));
@@ -399,6 +403,7 @@ Status 
VariantColumnReader::new_iterator(ColumnIteratorUPtr* iterator,
     } else {
         // Sparse column not exists and not reached stats limit, then the 
target path is not exist, get a default iterator
         RETURN_IF_ERROR(Segment::new_default_iterator(*target_col, iterator));
+        opt->stats->variant_subtree_default_iter_count++;
     }
     return Status::OK();
 }
diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp 
b/be/src/pipeline/exec/olap_scan_operator.cpp
index 9af5a97b5d6..1a33a3a7e77 100644
--- a/be/src/pipeline/exec/olap_scan_operator.cpp
+++ b/be/src/pipeline/exec/olap_scan_operator.cpp
@@ -324,6 +324,19 @@ Status OlapScanLocalState::_init_profile() {
     _ann_range_result_convert_costs =
             ADD_CHILD_TIMER(_segment_profile, 
"AnnIndexRangeResultConvertCosts",
                             "AnnIndexRangeResultPostProcessCosts");
+    _variant_scan_sparse_column_timer = ADD_TIMER(_segment_profile, 
"VariantScanSparseColumnTimer");
+    _variant_scan_sparse_column_bytes =
+            ADD_COUNTER(_segment_profile, "VariantScanSparseColumnBytes", 
TUnit::BYTES);
+    _variant_fill_path_from_sparse_column_timer =
+            ADD_TIMER(_segment_profile, 
"VariantFillPathFromSparseColumnTimer");
+    _variant_subtree_default_iter_count =
+            ADD_COUNTER(_segment_profile, "VariantSubtreeDefaultIterCount", 
TUnit::UNIT);
+    _variant_subtree_leaf_iter_count =
+            ADD_COUNTER(_segment_profile, "VariantSubtreeLeafIterCount", 
TUnit::UNIT);
+    _variant_subtree_hierarchical_iter_count =
+            ADD_COUNTER(_segment_profile, 
"VariantSubtreeHierarchicalIterCount", TUnit::UNIT);
+    _variant_subtree_sparse_iter_count =
+            ADD_COUNTER(_segment_profile, "VariantSubtreeSparseIterCount", 
TUnit::UNIT);
 
     return Status::OK();
 }
diff --git a/be/src/pipeline/exec/olap_scan_operator.h 
b/be/src/pipeline/exec/olap_scan_operator.h
index 7cb48e88d7c..f3b37c7fc89 100644
--- a/be/src/pipeline/exec/olap_scan_operator.h
+++ b/be/src/pipeline/exec/olap_scan_operator.h
@@ -269,6 +269,22 @@ private:
     RuntimeProfile::Counter* _segment_create_column_readers_timer = nullptr;
     RuntimeProfile::Counter* _segment_load_index_timer = nullptr;
 
+    // total uncompressed bytes read when scanning sparse columns in variant
+    RuntimeProfile::Counter* _variant_scan_sparse_column_bytes = nullptr;
+
+    // total time spent scanning sparse subcolumns
+    RuntimeProfile::Counter* _variant_scan_sparse_column_timer = nullptr;
+    // time to build/resolve subcolumn paths from the sparse column
+    RuntimeProfile::Counter* _variant_fill_path_from_sparse_column_timer = 
nullptr;
+    // Variant subtree: times falling back to default iterator due to missing 
path
+    RuntimeProfile::Counter* _variant_subtree_default_iter_count = nullptr;
+    // Variant subtree: times selecting leaf iterator (target subcolumn is a 
leaf)
+    RuntimeProfile::Counter* _variant_subtree_leaf_iter_count = nullptr;
+    // Variant subtree: times selecting hierarchical iterator (node has 
children and sparse columns)
+    RuntimeProfile::Counter* _variant_subtree_hierarchical_iter_count = 
nullptr;
+    // Variant subtree: times selecting sparse iterator (iterate over sparse 
subcolumn)
+    RuntimeProfile::Counter* _variant_subtree_sparse_iter_count = nullptr;
+
     std::vector<TabletWithVersion> _tablets;
     std::vector<TabletReader::ReadSource> _read_sources;
 
diff --git a/be/src/service/point_query_executor.cpp 
b/be/src/service/point_query_executor.cpp
index b5d91d39b1a..a6c7c7e527a 100644
--- a/be/src/service/point_query_executor.cpp
+++ b/be/src/service/point_query_executor.cpp
@@ -520,8 +520,12 @@ Status PointQueryExecutor::_lookup_row_data() {
                         
_result_block->get_by_position(pos).column->assume_mutable();
                 std::unique_ptr<ColumnIterator> iter;
                 SlotDescriptor* slot = _reusable->tuple_desc()->slots()[pos];
+                StorageReadOptions storage_read_options;
+                storage_read_options.stats = &_read_stats;
+                storage_read_options.io_ctx.reader_type = 
ReaderType::READER_QUERY;
                 
RETURN_IF_ERROR(segment->seek_and_read_by_rowid(*_tablet->tablet_schema(), slot,
-                                                                row_id, 
column, _read_stats, iter));
+                                                                row_id, column,
+                                                                
storage_read_options, iter));
                 if (_tablet->tablet_schema()
                             ->column_by_uid(slot->col_unique_id())
                             .has_char_type()) {
diff --git a/be/src/vec/exec/scan/olap_scanner.cpp 
b/be/src/vec/exec/scan/olap_scanner.cpp
index 120a370ae31..e3f8e06f3a6 100644
--- a/be/src/vec/exec/scan/olap_scanner.cpp
+++ b/be/src/vec/exec/scan/olap_scanner.cpp
@@ -747,6 +747,20 @@ void OlapScanner::_collect_profile_before_close() {
     COUNTER_UPDATE(local_state->_inverted_index_analyzer_timer,
                    stats.inverted_index_analyzer_timer);
     COUNTER_UPDATE(local_state->_inverted_index_lookup_timer, 
stats.inverted_index_lookup_timer);
+    COUNTER_UPDATE(local_state->_variant_scan_sparse_column_timer,
+                   stats.variant_scan_sparse_column_timer_ns);
+    COUNTER_UPDATE(local_state->_variant_scan_sparse_column_bytes,
+                   stats.variant_scan_sparse_column_bytes);
+    COUNTER_UPDATE(local_state->_variant_fill_path_from_sparse_column_timer,
+                   stats.variant_fill_path_from_sparse_column_timer_ns);
+    COUNTER_UPDATE(local_state->_variant_subtree_default_iter_count,
+                   stats.variant_subtree_default_iter_count);
+    COUNTER_UPDATE(local_state->_variant_subtree_leaf_iter_count,
+                   stats.variant_subtree_leaf_iter_count);
+    COUNTER_UPDATE(local_state->_variant_subtree_hierarchical_iter_count,
+                   stats.variant_subtree_hierarchical_iter_count);
+    COUNTER_UPDATE(local_state->_variant_subtree_sparse_iter_count,
+                   stats.variant_subtree_sparse_iter_count);
 
     InvertedIndexProfileReporter inverted_index_profile;
     inverted_index_profile.update(local_state->_index_filter_profile.get(),
diff --git 
a/be/test/olap/rowset/segment_v2/variant_column_writer_reader_test.cpp 
b/be/test/olap/rowset/segment_v2/variant_column_writer_reader_test.cpp
index e75a85f3051..420202b20dd 100644
--- a/be/test/olap/rowset/segment_v2/variant_column_writer_reader_test.cpp
+++ b/be/test/olap/rowset/segment_v2/variant_column_writer_reader_test.cpp
@@ -315,13 +315,14 @@ TEST_F(VariantColumnWriterReaderTest, 
test_write_data_normal) {
     ColumnIteratorUPtr it;
     TabletColumn parent_column = _tablet_schema->column(0);
     StorageReadOptions storage_read_opts;
+    OlapReaderStatistics stats;
     storage_read_opts.io_ctx.reader_type = ReaderType::READER_QUERY;
+    storage_read_opts.stats = &stats;
     st = variant_column_reader->new_iterator(&it, &parent_column, 
&storage_read_opts,
                                              &column_reader_cache);
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(assert_cast<HierarchicalDataIterator*>(it.get()) != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = it->init(column_iter_opts);
@@ -421,6 +422,7 @@ TEST_F(VariantColumnWriterReaderTest, 
test_write_data_normal) {
             // read with opt
             auto* iter = assert_cast<SparseColumnExtractIterator*>(it.get());
             StorageReadOptions storage_read_opts1;
+            storage_read_opts1.stats = &stats;
             storage_read_opts1.io_ctx.reader_type = ReaderType::READER_QUERY;
             iter->_read_opts = &storage_read_opts1;
             st = iter->next_batch(&nrows, new_column_object, nullptr);
@@ -841,12 +843,13 @@ TEST_F(VariantColumnWriterReaderTest, 
test_write_data_advanced) {
     TabletColumn parent_column = _tablet_schema->column(0);
     StorageReadOptions storage_read_opts;
     storage_read_opts.io_ctx.reader_type = ReaderType::READER_QUERY;
+    OlapReaderStatistics stats;
+    storage_read_opts.stats = &stats;
     st = variant_column_reader->new_iterator(&it, &parent_column, 
&storage_read_opts,
                                              &column_reader_cache);
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(assert_cast<HierarchicalDataIterator*>(it.get()) != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = it->init(column_iter_opts);
@@ -1137,13 +1140,14 @@ TEST_F(VariantColumnWriterReaderTest, 
test_write_data_nullable) {
     ColumnIteratorUPtr it;
     TabletColumn parent_column = _tablet_schema->column(0);
     StorageReadOptions storage_read_opts;
+    OlapReaderStatistics stats;
+    storage_read_opts.stats = &stats;
     storage_read_opts.io_ctx.reader_type = ReaderType::READER_QUERY;
     st = variant_column_reader->new_iterator(&it, &parent_column, 
&storage_read_opts,
                                              &column_reader_cache);
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(assert_cast<HierarchicalDataIterator*>(it.get()) != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = it->init(column_iter_opts);
@@ -1739,6 +1743,8 @@ TEST_F(VariantColumnWriterReaderTest, 
test_no_sub_in_sparse_column) {
     // 10. test hierarchical reader with empty statistics
     ColumnIteratorUPtr iterator;
     StorageReadOptions read_opts;
+    OlapReaderStatistics stats;
+    read_opts.stats = &stats;
     st = variant_reader->new_iterator(&iterator, &column, &read_opts, 
&column_reader_cache);
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(iterator != nullptr);
@@ -1880,6 +1886,8 @@ TEST_F(VariantColumnWriterReaderTest, 
test_prefix_in_sub_and_sparse) {
     // 10. test hierarchical reader with empty statistics
     ColumnIteratorUPtr iterator;
     StorageReadOptions read_opts;
+    OlapReaderStatistics stats;
+    read_opts.stats = &stats;
     st = variant_reader->new_iterator(&iterator, &column, &read_opts, 
&column_reader_cache);
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(iterator != nullptr);
@@ -2192,6 +2200,8 @@ TEST_F(VariantColumnWriterReaderTest, test_nested_iter) {
 
     StorageReadOptions storageReadOptions;
     storageReadOptions.io_ctx.reader_type = ReaderType::READER_QUERY;
+    OlapReaderStatistics stats;
+    storageReadOptions.stats = &stats;
 
     ColumnIteratorUPtr nested_column_iter;
     st = variant_column_reader->new_iterator(&nested_column_iter, 
&_tablet_schema->column(0),
@@ -2201,7 +2211,6 @@ TEST_F(VariantColumnWriterReaderTest, test_nested_iter) {
     auto* nested_iter = 
assert_cast<HierarchicalDataIterator*>(nested_column_iter.get());
     EXPECT_TRUE(nested_iter != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = nested_iter->init(column_iter_opts);
@@ -2346,6 +2355,8 @@ TEST_F(VariantColumnWriterReaderTest, 
test_nested_iter_nullable) {
 
     StorageReadOptions storageReadOptions;
     storageReadOptions.io_ctx.reader_type = ReaderType::READER_QUERY;
+    OlapReaderStatistics stats;
+    storageReadOptions.stats = &stats;
 
     ColumnIteratorUPtr nested_column_iter;
     MockColumnReaderCache column_reader_cache(footer, file_reader, 
_tablet_schema);
@@ -2357,7 +2368,6 @@ TEST_F(VariantColumnWriterReaderTest, 
test_nested_iter_nullable) {
     auto* nested_iter = 
assert_cast<HierarchicalDataIterator*>(nested_column_iter.get());
     EXPECT_TRUE(nested_iter != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = nested_iter->init(column_iter_opts);
@@ -2519,6 +2529,8 @@ TEST_F(VariantColumnWriterReaderTest, 
test_read_with_checksum) {
     subcolumn.set_is_nullable(true);
     _tablet_schema->append_column(subcolumn);
     storage_read_opts.io_ctx.reader_type = ReaderType::READER_QUERY;
+    OlapReaderStatistics stats;
+    storage_read_opts.stats = &stats;
     ColumnIteratorUPtr hierarchical_it;
     st = variant_column_reader->new_iterator(&hierarchical_it, &subcolumn, 
&storage_read_opts,
                                              &column_reader_cache);
@@ -2532,7 +2544,6 @@ TEST_F(VariantColumnWriterReaderTest, 
test_read_with_checksum) {
     EXPECT_TRUE(st.ok()) << st.msg();
     EXPECT_TRUE(dynamic_cast<FileColumnIterator*>(it.get()) != nullptr);
     ColumnIteratorOptions column_iter_opts;
-    OlapReaderStatistics stats;
     column_iter_opts.stats = &stats;
     column_iter_opts.file_reader = file_reader.get();
     st = it->init(column_iter_opts);


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to