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

dataroaring 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 65b2a5f0aa0 [Opt](sys table) Add a system table column_data_sizes 
(#56815)
65b2a5f0aa0 is described below

commit 65b2a5f0aa02d6a6765c156c288dd7ea8e0684c0
Author: bobhan1 <[email protected]>
AuthorDate: Thu Oct 16 10:40:26 2025 +0800

    [Opt](sys table) Add a system table column_data_sizes (#56815)
    
    ### What problem does this PR solve?
    
    Add a system table `column_data_sizes` for querying columns' data page
    sizes:
    ```sql
    MySQL [email protected]:information_schema> show create table 
information_schema.column_data_sizes;
    +-------------------+------------------------------------------+
    | Table             | Create Table                             |
    +-------------------+------------------------------------------+
    | column_data_sizes | CREATE TABLE `column_data_sizes` (       |
    |                   |   `BACKEND_ID` bigint NULL,              |
    |                   |   `TABLE_ID` bigint NULL,                |
    |                   |   `INDEX_ID` bigint NULL,                |
    |                   |   `PARTITION_ID` bigint NULL,            |
    |                   |   `TABLET_ID` bigint NULL,               |
    |                   |   `ROWSET_ID` varchar(64) NULL,          |
    |                   |   `COLUMN_UNIQUE_ID` int NULL,           |
    |                   |   `COLUMN_NAME` varchar(64) NULL,        |
    |                   |   `COLUMN_TYPE` varchar(64) NULL,        |
    |                   |   `COMPRESSED_DATA_BYTES` bigint NULL,   |
    |                   |   `UNCOMPRESSED_DATA_BYTES` bigint NULL, |
    |                   |   `RAW_DATA_BYTES` bigint NULL           |
    |                   | ) ENGINE=SCHEMA;                         |
    +-------------------+------------------------------------------+
    ```
    - **COMPRESSED_DATA_BYTES**: total size of compressed data pages in
    bytes, for complex type(array/map/struct), it's the sum of all leaf
    scalar column writers' stats
    - **UNCOMPRESSED_DATA_BYTES**: total size of uncompressed data pages in
    bytes, for complex type(array/map/struct), it's the sum of all leaf
    scalar column writers' stats
    - **RAW_DATA_BYTES**: total size of input raw data in bytes(calculated
    via `XXXPageBuilder`'s `Status add(const uint8_t* vals, size_t* count)`
    ), for complex type(array/map/struct), it's the sum of all leaf scalar
    column writers' stats
    
    To query columns' data page sizes in a table:
    ```sql
    SELECT
        COLUMN_NAME,
        sum(COMPRESSED_DATA_BYTES) AS compressed_data_bytes,
        sum(UNCOMPRESSED_DATA_BYTES) AS uncompressed_data_bytes,
        sum(RAW_DATA_BYTES) as raw_data_bytes
    FROM information_schema.column_data_sizes
    WHERE table_id=${TABLE_ID}
    GROUP BY COLUMN_NAME, COLUMN_TYPE
    ORDER BY data_size desc;
    ```
    
    Note:
    - variant column is not supported.
    - In cloud mode, only BE local's rowsets will be queried.
    
    ### Release note
    
    None
    
    ### Check List (For Author)
    
    - Test <!-- At least one of them must be included. -->
        - [ ] Regression test
        - [ ] Unit Test
        - [ ] Manual test (add detailed scripts or steps below)
        - [ ] No need to test or manual test. Explain why:
    - [ ] This is a refactor/code format and no logic has been changed.
            - [ ] Previous test can cover this change.
            - [ ] No code files have been changed.
            - [ ] Other reason <!-- Add your reason?  -->
    
    - Behavior changed:
        - [ ] No.
        - [ ] Yes. <!-- Explain the behavior change -->
    
    - Does this need documentation?
        - [ ] No.
    - [ ] Yes. <!-- Add document PR link here. eg:
    https://github.com/apache/doris-website/pull/1214 -->
    
    ### Check List (For Reviewer who merge this PR)
    
    - [ ] Confirm the release note
    - [ ] Confirm test cases
    - [ ] Confirm document
    - [ ] Add branch pick label <!-- Add branch pick label that this PR
    should merge into -->
---
 be/src/exec/schema_scanner.cpp                     |   3 +
 .../schema_column_data_sizes_scanner.cpp           | 363 +++++++++++++++++++++
 .../schema_column_data_sizes_scanner.h             |  71 ++++
 be/src/olap/rowset/segment_v2/binary_dict_page.cpp |  15 +-
 be/src/olap/rowset/segment_v2/binary_dict_page.h   |   3 +
 be/src/olap/rowset/segment_v2/binary_plain_page.h  |   5 +
 .../olap/rowset/segment_v2/binary_prefix_page.cpp  |   1 +
 be/src/olap/rowset/segment_v2/binary_prefix_page.h |   4 +
 be/src/olap/rowset/segment_v2/bitshuffle_page.h    |   5 +
 be/src/olap/rowset/segment_v2/column_writer.cpp    |  24 +-
 be/src/olap/rowset/segment_v2/column_writer.h      | 119 ++++++-
 .../rowset/segment_v2/frame_of_reference_page.h    |   5 +
 be/src/olap/rowset/segment_v2/page_builder.h       |   4 +
 be/src/olap/rowset/segment_v2/plain_page.h         |   5 +
 be/src/olap/rowset/segment_v2/rle_page.h           |   5 +
 be/src/olap/rowset/segment_v2/segment.cpp          |   9 +
 be/src/olap/rowset/segment_v2/segment.h            |   2 +
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |  14 +
 .../variant/variant_column_writer_impl.cpp         |   2 +-
 .../rowset/segment_v2/vertical_segment_writer.cpp  |  14 +
 .../org/apache/doris/analysis/SchemaTableType.java |   4 +-
 .../java/org/apache/doris/catalog/SchemaTable.java |  16 +
 gensrc/proto/segment_v2.proto                      |   4 +
 gensrc/thrift/Descriptors.thrift                   |   1 +
 .../system/test_query_sys_column_data_sizes.out    |  15 +
 .../system/test_query_sys_column_data_sizes.groovy | 130 ++++++++
 .../suites/show_p0/test_show_data_warehouse.groovy |  10 +-
 27 files changed, 837 insertions(+), 16 deletions(-)

diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index 3c734f2f4c2..370ea72495a 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -35,6 +35,7 @@
 #include "exec/schema_scanner/schema_cluster_snapshot_properties_scanner.h"
 #include "exec/schema_scanner/schema_cluster_snapshots_scanner.h"
 #include "exec/schema_scanner/schema_collations_scanner.h"
+#include "exec/schema_scanner/schema_column_data_sizes_scanner.h"
 #include "exec/schema_scanner/schema_columns_scanner.h"
 #include "exec/schema_scanner/schema_dummy_scanner.h"
 #include "exec/schema_scanner/schema_encryption_keys_scanner.h"
@@ -251,6 +252,8 @@ std::unique_ptr<SchemaScanner> 
SchemaScanner::create(TSchemaTableType::type type
         return SchemaClusterSnapshotsScanner::create_unique();
     case TSchemaTableType::SCH_CLUSTER_SNAPSHOT_PROPERTIES:
         return SchemaClusterSnapshotPropertiesScanner::create_unique();
+    case TSchemaTableType::SCH_COLUMN_DATA_SIZES:
+        return SchemaColumnDataSizesScanner::create_unique();
     default:
         return SchemaDummyScanner::create_unique();
         break;
diff --git a/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.cpp 
b/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.cpp
new file mode 100644
index 00000000000..7ad811f6489
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.cpp
@@ -0,0 +1,363 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/schema_scanner/schema_column_data_sizes_scanner.h"
+
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/segment_v2.pb.h>
+
+#include <algorithm>
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <shared_mutex>
+#include <string>
+#include <utility>
+
+#include "cloud/cloud_storage_engine.h"
+#include "cloud/cloud_tablet.h"
+#include "cloud/cloud_tablet_mgr.h"
+#include "cloud/config.h"
+#include "common/status.h"
+#include "olap/olap_common.h"
+#include "olap/rowset/beta_rowset.h"
+#include "olap/rowset/rowset.h"
+#include "olap/rowset/rowset_meta.h"
+#include "olap/rowset/segment_v2/segment.h"
+#include "olap/storage_engine.h"
+#include "olap/tablet.h"
+#include "olap/tablet_manager.h"
+#include "runtime/define_primitive_type.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "util/runtime_profile.h"
+#include "vec/common/string_ref.h"
+
+namespace doris {
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+#include "common/compile_check_begin.h"
+
+std::vector<SchemaScanner::ColumnDesc> 
SchemaColumnDataSizesScanner::_s_tbls_columns = {
+        //   name,                type,          size,             is_null
+        {"BACKEND_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"TABLE_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"INDEX_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"PARTITION_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"TABLET_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"ROWSET_ID", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COLUMN_UNIQUE_ID", TYPE_INT, sizeof(int32_t), true},
+        {"COLUMN_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COLUMN_TYPE", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"COMPRESSED_DATA_BYTES", TYPE_BIGINT, sizeof(int64_t), true},
+        {"UNCOMPRESSED_DATA_BYTES", TYPE_BIGINT, sizeof(int64_t), true},
+        {"RAW_DATA_BYTES", TYPE_BIGINT, sizeof(int64_t), true}};
+
+SchemaColumnDataSizesScanner::SchemaColumnDataSizesScanner()
+        : SchemaScanner(_s_tbls_columns, 
TSchemaTableType::SCH_COLUMN_DATA_SIZES),
+          backend_id_(0),
+          _column_data_sizes_idx(0) {}
+
+SchemaColumnDataSizesScanner::~SchemaColumnDataSizesScanner() = default;
+
+Status SchemaColumnDataSizesScanner::start(RuntimeState* state) {
+    if (!_is_init) {
+        return Status::InternalError("used before initialized.");
+    }
+    backend_id_ = state->backend_id();
+    RETURN_IF_ERROR(_get_all_column_data_sizes());
+    return Status::OK();
+}
+
+Status SchemaColumnDataSizesScanner::_get_all_column_data_sizes() {
+    auto process_rowsets = [&](const std::vector<RowsetSharedPtr>& rowsets, 
int64_t table_id,
+                               int64_t index_id, int64_t partition_id, int64_t 
tablet_id) {
+        for (const auto& rowset : rowsets) {
+            auto beta_rowset = std::dynamic_pointer_cast<BetaRowset>(rowset);
+            if (!beta_rowset) {
+                continue;
+            }
+
+            if (beta_rowset->num_segments() == 0) {
+                continue;
+            }
+
+            auto rowset_meta = rowset->rowset_meta();
+            const auto& schema = rowset_meta->tablet_schema();
+            auto rowset_id = rowset_meta->rowset_id().to_string();
+
+            std::map<int32_t /* column_unique_id */, ColumnDataSizeInfo> 
aggregated_stats;
+
+            // Load all segments at once
+            std::vector<segment_v2::SegmentSharedPtr> segments;
+            auto st = beta_rowset->load_segments(&segments);
+            if (!st.ok()) {
+                LOG(WARNING) << "Failed to load segments for rowset "
+                             << beta_rowset->rowset_id().to_string()
+                             << ", error: " << st.to_string();
+                continue;
+            }
+
+            // Get column data page stats from each segment footer and 
aggregate by column_unique_id
+            for (const auto& segment : segments) {
+                auto collector = [&](const segment_v2::ColumnMetaPB& 
column_meta) {
+                    if (column_meta.has_compressed_data_bytes() &&
+                        column_meta.has_uncompressed_data_bytes()) {
+                        auto cid = 
schema->field_index(column_meta.unique_id());
+                        if (cid == -1) {
+                            return;
+                        }
+                        // Aggregate stats by column_unique_id
+                        if 
(aggregated_stats.contains(column_meta.unique_id())) {
+                            auto& existing_stats = 
aggregated_stats[column_meta.unique_id()];
+                            existing_stats.compressed_data_bytes +=
+                                    column_meta.compressed_data_bytes();
+                            existing_stats.uncompressed_data_bytes +=
+                                    column_meta.uncompressed_data_bytes();
+                            existing_stats.raw_data_bytes += 
column_meta.raw_data_bytes();
+                        } else {
+                            aggregated_stats[column_meta.unique_id()] = 
ColumnDataSizeInfo {
+                                    .backend_id = backend_id_,
+                                    .table_id = table_id,
+                                    .index_id = index_id,
+                                    .partition_id = partition_id,
+                                    .tablet_id = tablet_id,
+                                    .rowset_id = rowset_id,
+                                    .column_unique_id = 
column_meta.unique_id(),
+                                    .column_name = schema->column(cid).name(),
+                                    .column_type = 
TabletColumn::get_string_by_field_type(
+                                            
static_cast<FieldType>(column_meta.type())),
+                                    .compressed_data_bytes = 
column_meta.compressed_data_bytes(),
+                                    .uncompressed_data_bytes =
+                                            
column_meta.uncompressed_data_bytes(),
+                                    .raw_data_bytes = 
column_meta.raw_data_bytes(),
+                            };
+                        }
+                    }
+                };
+                st = segment->traverse_column_meta_pbs(collector);
+                if (!st.ok()) {
+                    continue;
+                }
+            }
+            // Append aggregated stats for this rowset to the result vector
+            for (const auto& [_, stats] : aggregated_stats) {
+                _column_data_sizes.push_back(stats);
+            }
+        }
+    };
+
+    if (config::is_cloud_mode()) {
+        // only query cloud tablets in lru cache instead of all tablets
+        std::vector<std::weak_ptr<CloudTablet>> tablets =
+                
ExecEnv::GetInstance()->storage_engine().to_cloud().tablet_mgr().get_weak_tablets();
+        for (const std::weak_ptr<CloudTablet>& tablet : tablets) {
+            if (!tablet.expired()) {
+                auto t = tablet.lock();
+                std::vector<RowsetSharedPtr> rowsets;
+                {
+                    std::shared_lock rowset_ldlock(t->get_header_lock());
+                    for (const auto& it : t->rowset_map()) {
+                        rowsets.emplace_back(it.second);
+                    }
+                }
+                process_rowsets(rowsets, t->table_id(), t->index_id(), 
t->partition_id(),
+                                t->tablet_id());
+            }
+        }
+    } else {
+        std::vector<TabletSharedPtr> tablets = ExecEnv::GetInstance()
+                                                       ->storage_engine()
+                                                       .to_local()
+                                                       .tablet_manager()
+                                                       ->get_all_tablet();
+        for (const auto& tablet : tablets) {
+            std::vector<std::pair<Version, RowsetSharedPtr>> all_rowsets;
+            {
+                std::shared_lock rowset_rlock(tablet->get_header_lock());
+                tablet->acquire_version_and_rowsets(&all_rowsets);
+            }
+            std::vector<RowsetSharedPtr> rowsets;
+            for (const auto& version_and_rowset : all_rowsets) {
+                rowsets.emplace_back(version_and_rowset.second);
+            }
+            process_rowsets(rowsets, tablet->table_id(), tablet->index_id(), 
tablet->partition_id(),
+                            tablet->tablet_id());
+        }
+    }
+    return Status::OK();
+}
+
+Status 
SchemaColumnDataSizesScanner::get_next_block_internal(vectorized::Block* block, 
bool* eos) {
+    if (!_is_init) {
+        return Status::InternalError("Used before initialized.");
+    }
+    if (nullptr == block || nullptr == eos) {
+        return Status::InternalError("input pointer is nullptr.");
+    }
+
+    if (_column_data_sizes_idx >= _column_data_sizes.size()) {
+        *eos = true;
+        return Status::OK();
+    }
+    *eos = false;
+    return _fill_block_impl(block);
+}
+
+Status SchemaColumnDataSizesScanner::_fill_block_impl(vectorized::Block* 
block) {
+    SCOPED_TIMER(_fill_block_timer);
+    size_t fill_num = std::min(1000UL, _column_data_sizes.size() - 
_column_data_sizes_idx);
+    size_t fill_idx_begin = _column_data_sizes_idx;
+    size_t fill_idx_end = _column_data_sizes_idx + fill_num;
+    std::vector<void*> datas(fill_num);
+
+    // BACKEND_ID
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].backend_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 0, datas));
+    }
+
+    // TABLE_ID
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].table_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 1, datas));
+    }
+
+    // INDEX_ID
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].index_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 2, datas));
+    }
+
+    // PARTITION_ID
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].partition_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 3, datas));
+    }
+
+    // TABLET_ID
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].tablet_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 4, datas));
+    }
+
+    // ROWSET_ID
+    {
+        std::vector<std::string> rowset_ids(fill_num);
+        std::vector<StringRef> strs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            rowset_ids[i - fill_idx_begin] = _column_data_sizes[i].rowset_id;
+            strs[i - fill_idx_begin] = StringRef(rowset_ids[i - 
fill_idx_begin].c_str(),
+                                                 rowset_ids[i - 
fill_idx_begin].size());
+            datas[i - fill_idx_begin] = strs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 5, datas));
+    }
+
+    // COLUMN_UNIQUE_ID
+    {
+        std::vector<int32_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].column_unique_id;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 6, datas));
+    }
+
+    // COLUMN_NAME
+    {
+        std::vector<std::string> column_names(fill_num);
+        std::vector<StringRef> strs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            column_names[i - fill_idx_begin] = 
_column_data_sizes[i].column_name;
+            strs[i - fill_idx_begin] = StringRef(column_names[i - 
fill_idx_begin].c_str(),
+                                                 column_names[i - 
fill_idx_begin].size());
+            datas[i - fill_idx_begin] = strs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 7, datas));
+    }
+
+    // COLUMN_TYPE
+    {
+        std::vector<std::string> column_types(fill_num);
+        std::vector<StringRef> strs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            column_types[i - fill_idx_begin] = 
_column_data_sizes[i].column_type;
+            strs[i - fill_idx_begin] = StringRef(column_types[i - 
fill_idx_begin].c_str(),
+                                                 column_types[i - 
fill_idx_begin].size());
+            datas[i - fill_idx_begin] = strs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 8, datas));
+    }
+
+    // COMPRESSED_DATA_SIZE
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = 
_column_data_sizes[i].compressed_data_bytes;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 9, datas));
+    }
+
+    // UNCOMPRESSED_DATA_SIZE
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = 
_column_data_sizes[i].uncompressed_data_bytes;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 10, datas));
+    }
+
+    // RAW_DATA_SIZE
+    {
+        std::vector<int64_t> srcs(fill_num);
+        for (size_t i = fill_idx_begin; i < fill_idx_end; ++i) {
+            srcs[i - fill_idx_begin] = _column_data_sizes[i].raw_data_bytes;
+            datas[i - fill_idx_begin] = srcs.data() + i - fill_idx_begin;
+        }
+        RETURN_IF_ERROR(fill_dest_column_for_range(block, 11, datas));
+    }
+
+    _column_data_sizes_idx += fill_num;
+    return Status::OK();
+}
+
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.h 
b/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.h
new file mode 100644
index 00000000000..bcf1363ceb1
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_column_data_sizes_scanner.h
@@ -0,0 +1,71 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <gen_cpp/Descriptors_types.h>
+
+#include <memory>
+#include <vector>
+
+#include "exec/schema_scanner.h"
+
+namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+class SchemaColumnDataSizesScanner : public SchemaScanner {
+public:
+    SchemaColumnDataSizesScanner();
+    ~SchemaColumnDataSizesScanner() override;
+
+    Status start(RuntimeState* state) override;
+    Status get_next_block_internal(vectorized::Block* block, bool* eos) 
override;
+
+    static std::unique_ptr<SchemaScanner> create_unique() {
+        return std::make_unique<SchemaColumnDataSizesScanner>();
+    }
+
+private:
+    Status _get_all_column_data_sizes();
+    Status _fill_block_impl(vectorized::Block* block);
+
+    struct ColumnDataSizeInfo {
+        int64_t backend_id;
+        int64_t table_id;
+        int64_t index_id;
+        int64_t partition_id;
+        int64_t tablet_id;
+        std::string rowset_id;
+        uint32_t column_unique_id;
+        std::string column_name;
+        std::string column_type;
+        uint64_t compressed_data_bytes;
+        uint64_t uncompressed_data_bytes;
+        uint64_t raw_data_bytes;
+    };
+
+    static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;
+
+    int64_t backend_id_;
+    std::vector<ColumnDataSizeInfo> _column_data_sizes;
+    size_t _column_data_sizes_idx;
+};
+
+} // namespace doris
\ No newline at end of file
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp 
b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
index a61c60474fc..f1c1babb26a 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.cpp
@@ -131,13 +131,21 @@ Status BinaryDictPageBuilder::add(const uint8_t* vals, 
size_t* count) {
                 // current data page is full, stop processing remaining inputs
                 break;
             }
+            // Track raw data size: the original string size
+            _raw_data_size += src->size;
             num_added += 1;
         }
         *count = num_added;
         return Status::OK();
     } else {
         DCHECK_EQ(_encoding_type, PLAIN_ENCODING);
-        return _data_page_builder->add(vals, count);
+        RETURN_IF_ERROR(_data_page_builder->add(vals, count));
+        // For plain encoding, track raw data size from the input
+        const Slice* src = reinterpret_cast<const Slice*>(vals);
+        for (size_t i = 0; i < *count; ++i) {
+            _raw_data_size += src[i].size;
+        }
+        return Status::OK();
     }
 }
 
@@ -162,6 +170,7 @@ Status BinaryDictPageBuilder::finish(OwnedSlice* slice) {
 Status BinaryDictPageBuilder::reset() {
     RETURN_IF_CATCH_EXCEPTION({
         _finished = false;
+        _raw_data_size = 0;
         _buffer.reserve(_options.data_page_size + 
BINARY_DICT_PAGE_HEADER_SIZE);
         _buffer.resize(BINARY_DICT_PAGE_HEADER_SIZE);
 
@@ -216,6 +225,10 @@ Status BinaryDictPageBuilder::get_last_value(void* value) 
const {
     return Status::OK();
 }
 
+uint64_t BinaryDictPageBuilder::get_raw_data_size() const {
+    return _raw_data_size;
+}
+
 BinaryDictPageDecoder::BinaryDictPageDecoder(Slice data, const 
PageDecoderOptions& options)
         : _data(data),
           _options(options),
diff --git a/be/src/olap/rowset/segment_v2/binary_dict_page.h 
b/be/src/olap/rowset/segment_v2/binary_dict_page.h
index f591d299396..ee42881d4c4 100644
--- a/be/src/olap/rowset/segment_v2/binary_dict_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_dict_page.h
@@ -82,6 +82,8 @@ public:
 
     Status get_last_value(void* value) const override;
 
+    uint64_t get_raw_data_size() const override;
+
 private:
     BinaryDictPageBuilder(const PageBuilderOptions& options);
 
@@ -103,6 +105,7 @@ private:
     vectorized::Arena _arena;
     faststring _buffer;
     faststring _first_value;
+    uint64_t _raw_data_size = 0;
 
     bool _has_empty = false;
     uint32_t _empty_code = 0;
diff --git a/be/src/olap/rowset/segment_v2/binary_plain_page.h 
b/be/src/olap/rowset/segment_v2/binary_plain_page.h
index 6ba4594a142..e552fcda266 100644
--- a/be/src/olap/rowset/segment_v2/binary_plain_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_plain_page.h
@@ -84,6 +84,7 @@ public:
             _last_value_size = cast_set<uint32_t>(src->size);
             _size_estimate += src->size;
             _size_estimate += sizeof(uint32_t);
+            _raw_data_size += src->size;
 
             i++;
             vals += sizeof(Slice);
@@ -121,6 +122,7 @@ public:
             _size_estimate = sizeof(uint32_t);
             _finished = false;
             _last_value_size = 0;
+            _raw_data_size = 0;
         });
         return Status::OK();
     }
@@ -129,6 +131,8 @@ public:
 
     uint64_t size() const override { return _size_estimate; }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         DCHECK(_finished);
         if (_offsets.size() == 0) {
@@ -174,6 +178,7 @@ private:
     PageBuilderOptions _options;
     // size of last added value
     uint32_t _last_value_size = 0;
+    uint64_t _raw_data_size = 0;
     faststring _first_value;
     faststring _last_value;
 };
diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp 
b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
index 2b63b5ae5e7..a9fa6082111 100644
--- a/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
+++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.cpp
@@ -79,6 +79,7 @@ Status BinaryPrefixPageBuilder::add(const uint8_t* vals, 
size_t* add_count) {
             _last_entry.append(entry, entry_len);
         });
 
+        _raw_data_size += entry_len;
         ++_count;
     }
     *add_count = i;
diff --git a/be/src/olap/rowset/segment_v2/binary_prefix_page.h 
b/be/src/olap/rowset/segment_v2/binary_prefix_page.h
index 41deb4e6c1f..22c81f0ef59 100644
--- a/be/src/olap/rowset/segment_v2/binary_prefix_page.h
+++ b/be/src/olap/rowset/segment_v2/binary_prefix_page.h
@@ -60,6 +60,7 @@ public:
         _count = 0;
         _buffer.clear();
         _finished = false;
+        _raw_data_size = 0;
         return Status::OK();
     }
 
@@ -73,6 +74,8 @@ public:
 
     size_t count() const override { return _count; }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         DCHECK(_finished);
         if (_count == 0) {
@@ -101,6 +104,7 @@ private:
     size_t _count = 0;
     bool _finished = false;
     faststring _buffer;
+    uint64_t _raw_data_size = 0;
     // This is a empirical value, Kudu and LevelDB use this default value
     static const uint8_t RESTART_POINT_INTERVAL = 16;
 };
diff --git a/be/src/olap/rowset/segment_v2/bitshuffle_page.h 
b/be/src/olap/rowset/segment_v2/bitshuffle_page.h
index 25528261ce8..f518ac655d7 100644
--- a/be/src/olap/rowset/segment_v2/bitshuffle_page.h
+++ b/be/src/olap/rowset/segment_v2/bitshuffle_page.h
@@ -133,6 +133,7 @@ public:
         RETURN_IF_CATCH_EXCEPTION(_data.resize(orig_size + to_add_size));
         _count += to_add;
         _remain_element_capacity -= to_add;
+        _raw_data_size += to_add_size;
         // return added number through count
         *num_written = to_add;
         if constexpr (single) {
@@ -171,6 +172,7 @@ public:
         RETURN_IF_CATCH_EXCEPTION({
             size_t block_size = _options.data_page_size;
             _count = 0;
+            _raw_data_size = 0;
             _data.clear();
             _data.reserve(block_size);
             DCHECK_EQ(reinterpret_cast<uintptr_t>(_data.data()) & 
(alignof(CppType) - 1), 0)
@@ -187,6 +189,8 @@ public:
 
     uint64_t size() const override { return _buffer.size(); }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         DCHECK(_finished);
         if (_count == 0) {
@@ -264,6 +268,7 @@ private:
     faststring _buffer;
     CppType _first_value;
     CppType _last_value;
+    uint64_t _raw_data_size = 0;
 };
 
 inline Status parse_bit_shuffle_header(const Slice& data, size_t& num_elements,
diff --git a/be/src/olap/rowset/segment_v2/column_writer.cpp 
b/be/src/olap/rowset/segment_v2/column_writer.cpp
index 461a5046e71..7175196ed20 100644
--- a/be/src/olap/rowset/segment_v2/column_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/column_writer.cpp
@@ -407,7 +407,7 @@ Status ColumnWriter::append(const uint8_t* nullmap, const 
void* data, size_t num
 
 ScalarColumnWriter::ScalarColumnWriter(const ColumnWriterOptions& opts,
                                        std::unique_ptr<Field> field, 
io::FileWriter* file_writer)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta),
           _opts(opts),
           _file_writer(file_writer),
           _data_size(0) {
@@ -615,7 +615,13 @@ Status ScalarColumnWriter::finish() {
 }
 
 Status ScalarColumnWriter::write_data() {
+    auto offset = _file_writer->bytes_appended();
+    auto collect_uncompressed_bytes = [](const PageFooterPB& footer) {
+        return footer.uncompressed_size() + footer.ByteSizeLong() +
+               sizeof(uint32_t) /* footer size */ + sizeof(uint32_t) /* 
checksum */;
+    };
     for (auto& page : _pages) {
+        _total_uncompressed_data_pages_size += 
collect_uncompressed_bytes(page->footer);
         RETURN_IF_ERROR(_write_data_page(page.get()));
     }
     _pages.clear();
@@ -628,6 +634,7 @@ Status ScalarColumnWriter::write_data() {
         footer.set_type(DICTIONARY_PAGE);
         
footer.set_uncompressed_size(cast_set<uint32_t>(dict_body.slice().get_size()));
         footer.mutable_dict_page_footer()->set_encoding(PLAIN_ENCODING);
+        _total_uncompressed_data_pages_size += 
collect_uncompressed_bytes(footer);
 
         PagePointer dict_pp;
         RETURN_IF_ERROR(PageIO::compress_and_write_page(
@@ -635,6 +642,7 @@ Status ScalarColumnWriter::write_data() {
                 {dict_body.slice()}, footer, &dict_pp));
         dict_pp.to_proto(_opts.meta->mutable_dict_page());
     }
+    _total_compressed_data_pages_size += _file_writer->bytes_appended() - 
offset;
     _page_builder.reset();
     return Status::OK();
 }
@@ -700,6 +708,8 @@ Status ScalarColumnWriter::finish_current_page() {
         RETURN_IF_ERROR(_bloom_filter_index_builder->flush());
     }
 
+    _raw_data_bytes += _page_builder->get_raw_data_size();
+
     // build data page body : encoded values + [nullmap]
     std::vector<Slice> body;
     OwnedSlice encoded_values;
@@ -792,7 +802,7 @@ StructColumnWriter::StructColumnWriter(
         const ColumnWriterOptions& opts, std::unique_ptr<Field> field,
         ScalarColumnWriter* null_writer,
         std::vector<std::unique_ptr<ColumnWriter>>& sub_column_writers)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()), 
_opts(opts) {
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta), 
_opts(opts) {
     for (auto& sub_column_writer : sub_column_writers) {
         _sub_column_writers.push_back(std::move(sub_column_writer));
     }
@@ -901,7 +911,7 @@ ArrayColumnWriter::ArrayColumnWriter(const 
ColumnWriterOptions& opts, std::uniqu
                                      OffsetColumnWriter* offset_writer,
                                      ScalarColumnWriter* null_writer,
                                      std::unique_ptr<ColumnWriter> item_writer)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()),
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta),
           _item_writer(std::move(item_writer)),
           _opts(opts) {
     _offset_writer.reset(offset_writer);
@@ -985,7 +995,8 @@ Status ArrayColumnWriter::append_data(const uint8_t** ptr, 
size_t num_rows) {
                     reinterpret_cast<const uint8_t*>(nested_null_map), 
offsets_ptr, num_rows));
         } else {
             return Status::NotSupported(
-                    "Ann index can only be build on array with scalar type. 
but got {} as nested",
+                    "Ann index can only be build on array with scalar type. 
but got {} as "
+                    "nested",
                     _item_writer->get_field()->type());
         }
     }
@@ -1073,7 +1084,7 @@ Status ArrayColumnWriter::finish_current_page() {
 MapColumnWriter::MapColumnWriter(const ColumnWriterOptions& opts, 
std::unique_ptr<Field> field,
                                  ScalarColumnWriter* null_writer, 
OffsetColumnWriter* offset_writer,
                                  std::vector<std::unique_ptr<ColumnWriter>>& 
kv_writers)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()), 
_opts(opts) {
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta), 
_opts(opts) {
     CHECK_EQ(kv_writers.size(), 2);
     _offsets_writer.reset(offset_writer);
     if (is_nullable()) {
@@ -1210,7 +1221,7 @@ Status MapColumnWriter::write_inverted_index() {
 
 VariantColumnWriter::VariantColumnWriter(const ColumnWriterOptions& opts,
                                          const TabletColumn* column, 
std::unique_ptr<Field> field)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()) {
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta) {
     _impl = std::make_unique<VariantColumnWriterImpl>(opts, column);
 }
 
@@ -1250,6 +1261,7 @@ Status VariantColumnWriter::write_inverted_index() {
 Status VariantColumnWriter::write_bloom_filter_index() {
     return _impl->write_bloom_filter_index();
 }
+
 Status VariantColumnWriter::append_nullable(const uint8_t* null_map, const 
uint8_t** ptr,
                                             size_t num_rows) {
     return _impl->append_nullable(null_map, ptr, num_rows);
diff --git a/be/src/olap/rowset/segment_v2/column_writer.h 
b/be/src/olap/rowset/segment_v2/column_writer.h
index 4f42d6bb750..05955840bac 100644
--- a/be/src/olap/rowset/segment_v2/column_writer.h
+++ b/be/src/olap/rowset/segment_v2/column_writer.h
@@ -121,8 +121,8 @@ public:
                                           const TabletColumn* column, 
io::FileWriter* file_writer,
                                           std::unique_ptr<ColumnWriter>* 
writer);
 
-    explicit ColumnWriter(std::unique_ptr<Field> field, bool is_nullable)
-            : _field(std::move(field)), _is_nullable(is_nullable) {}
+    explicit ColumnWriter(std::unique_ptr<Field> field, bool is_nullable, 
ColumnMetaPB* meta)
+            : _field(std::move(field)), _is_nullable(is_nullable), 
_column_meta(meta) {}
 
     virtual ~ColumnWriter() = default;
 
@@ -181,6 +181,10 @@ public:
 
     virtual ordinal_t get_next_rowid() const = 0;
 
+    virtual uint64_t get_raw_data_bytes() const = 0;
+    virtual uint64_t get_total_uncompressed_data_pages_bytes() const = 0;
+    virtual uint64_t get_total_compressed_data_pages_bytes() const = 0;
+
     // used for append not null data.
     virtual Status append_data(const uint8_t** ptr, size_t num_rows) = 0;
 
@@ -188,9 +192,12 @@ public:
 
     Field* get_field() const { return _field.get(); }
 
+    ColumnMetaPB* get_column_meta() const { return _column_meta; }
+
 private:
     std::unique_ptr<Field> _field;
     bool _is_nullable;
+    ColumnMetaPB* _column_meta;
     std::vector<uint8_t> _null_bitmap;
 };
 
@@ -230,6 +237,16 @@ public:
     Status write_bloom_filter_index() override;
     ordinal_t get_next_rowid() const override { return _next_rowid; }
 
+    uint64_t get_raw_data_bytes() const override { return _raw_data_bytes; }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return _total_uncompressed_data_pages_size;
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return _total_compressed_data_pages_size;
+    }
+
     void register_flush_page_callback(FlushPageCallback* flush_page_callback) {
         _new_page_callback = flush_page_callback;
     }
@@ -287,6 +304,10 @@ private:
     // total size of data page list
     uint64_t _data_size;
 
+    uint64_t _raw_data_bytes {0};
+    uint64_t _total_uncompressed_data_pages_size {0};
+    uint64_t _total_compressed_data_pages_size {0};
+
     // cached generated pages,
     std::vector<std::unique_ptr<Page>> _pages;
     ordinal_t _first_rowid = 0;
@@ -367,6 +388,28 @@ public:
 
     ordinal_t get_next_rowid() const override { return 
_sub_column_writers[0]->get_next_rowid(); }
 
+    uint64_t get_raw_data_bytes() const override {
+        return _get_total_data_pages_bytes(&ColumnWriter::get_raw_data_bytes);
+    }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_uncompressed_data_pages_bytes);
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_compressed_data_pages_bytes);
+    }
+
+private:
+    template <typename Func>
+    uint64_t _get_total_data_pages_bytes(Func func) const {
+        uint64_t size = is_nullable() ? std::invoke(func, _null_writer.get()) 
: 0;
+        for (const auto& writer : _sub_column_writers) {
+            size += std::invoke(func, writer.get());
+        }
+        return size;
+    }
+
 private:
     size_t _num_sub_column_writers;
     std::unique_ptr<ScalarColumnWriter> _null_writer;
@@ -418,6 +461,29 @@ public:
     }
     ordinal_t get_next_rowid() const override { return 
_offset_writer->get_next_rowid(); }
 
+    uint64_t get_raw_data_bytes() const override {
+        return _get_total_data_pages_bytes(&ColumnWriter::get_raw_data_bytes);
+    }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_uncompressed_data_pages_bytes);
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_compressed_data_pages_bytes);
+    }
+
+private:
+    template <typename Func>
+    uint64_t _get_total_data_pages_bytes(Func func) const {
+        uint64_t size = std::invoke(func, _offset_writer.get());
+        if (is_nullable()) {
+            size += std::invoke(func, _null_writer.get());
+        }
+        size += std::invoke(func, _item_writer.get());
+        return size;
+    }
+
 private:
     Status write_null_column(size_t num_rows, bool is_null); // 
写入num_rows个null标记
     bool has_empty_items() const { return _item_writer->get_next_rowid() == 0; 
}
@@ -476,6 +542,31 @@ public:
     // according key writer to get next rowid
     ordinal_t get_next_rowid() const override { return 
_offsets_writer->get_next_rowid(); }
 
+    uint64_t get_raw_data_bytes() const override {
+        return _get_total_data_pages_bytes(&ColumnWriter::get_raw_data_bytes);
+    }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_uncompressed_data_pages_bytes);
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return 
_get_total_data_pages_bytes(&ColumnWriter::get_total_compressed_data_pages_bytes);
+    }
+
+private:
+    template <typename Func>
+    uint64_t _get_total_data_pages_bytes(Func func) const {
+        uint64_t size = std::invoke(func, _offsets_writer.get());
+        if (is_nullable()) {
+            size += std::invoke(func, _null_writer.get());
+        }
+        for (const auto& writer : _kv_writers) {
+            size += std::invoke(func, writer.get());
+        }
+        return size;
+    }
+
 private:
     std::vector<std::unique_ptr<ColumnWriter>> _kv_writers;
     // we need null writer to make sure a row is null or not
@@ -510,6 +601,18 @@ public:
     Status write_bloom_filter_index() override;
     ordinal_t get_next_rowid() const override { return _next_rowid; }
 
+    uint64_t get_raw_data_bytes() const override {
+        return 0; // TODO
+    }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return 0; // TODO
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return 0; // TODO
+    }
+
     Status append_nulls(size_t num_rows) override {
         return Status::NotSupported("variant writer can not append_nulls");
     }
@@ -559,6 +662,18 @@ public:
     Status write_bloom_filter_index() override;
     ordinal_t get_next_rowid() const override { return _next_rowid; }
 
+    uint64_t get_raw_data_bytes() const override {
+        return 0; // TODO
+    }
+
+    uint64_t get_total_uncompressed_data_pages_bytes() const override {
+        return 0; // TODO
+    }
+
+    uint64_t get_total_compressed_data_pages_bytes() const override {
+        return 0; // TODO
+    }
+
     Status append_nulls(size_t num_rows) override {
         return Status::NotSupported("variant writer can not append_nulls");
     }
diff --git a/be/src/olap/rowset/segment_v2/frame_of_reference_page.h 
b/be/src/olap/rowset/segment_v2/frame_of_reference_page.h
index 495b9f8ca02..8bc9eaa9f74 100644
--- a/be/src/olap/rowset/segment_v2/frame_of_reference_page.h
+++ b/be/src/olap/rowset/segment_v2/frame_of_reference_page.h
@@ -51,6 +51,7 @@ public:
         }
         _encoder->put_batch(new_vals, *count);
         _count += *count;
+        _raw_data_size += *count * sizeof(CppType);
         _last_val = new_vals[*count - 1];
         return Status::OK();
     }
@@ -66,6 +67,7 @@ public:
     Status reset() override {
         _count = 0;
         _finished = false;
+        _raw_data_size = 0;
         _encoder->clear();
         return Status::OK();
     }
@@ -74,6 +76,8 @@ public:
 
     uint64_t size() const override { return _buf.size(); }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         if (_count == 0) {
             return Status::Error<ErrorCode::ENTRY_NOT_FOUND>("page is empty");
@@ -102,6 +106,7 @@ private:
     faststring _buf;
     CppType _first_val;
     CppType _last_val;
+    uint64_t _raw_data_size = 0;
 };
 
 template <FieldType Type>
diff --git a/be/src/olap/rowset/segment_v2/page_builder.h 
b/be/src/olap/rowset/segment_v2/page_builder.h
index b957c2399bd..5813b226411 100644
--- a/be/src/olap/rowset/segment_v2/page_builder.h
+++ b/be/src/olap/rowset/segment_v2/page_builder.h
@@ -83,6 +83,10 @@ public:
     // Return the total bytes of pageBuilder that have been added to the page.
     virtual uint64_t size() const = 0;
 
+    // Return the uncompressed data size in bytes (raw data added via add() 
method).
+    // This is used to track the original data size before compression.
+    virtual uint64_t get_raw_data_size() const = 0;
+
     // Return the first value in this page.
     // This method could only be called between finish() and reset().
     // Status::Error<ENTRY_NOT_FOUND> if no values have been added.
diff --git a/be/src/olap/rowset/segment_v2/plain_page.h 
b/be/src/olap/rowset/segment_v2/plain_page.h
index 1e9b12db332..1a537e5b8b7 100644
--- a/be/src/olap/rowset/segment_v2/plain_page.h
+++ b/be/src/olap/rowset/segment_v2/plain_page.h
@@ -58,6 +58,7 @@ public:
         RETURN_IF_CATCH_EXCEPTION(_buffer.resize(old_size + *count * 
SIZE_OF_TYPE));
         memcpy(&_buffer[old_size], vals, *count * SIZE_OF_TYPE);
         _count += *count;
+        _raw_data_size += *count * SIZE_OF_TYPE;
         return Status::OK();
     }
 
@@ -79,6 +80,7 @@ public:
         RETURN_IF_CATCH_EXCEPTION({
             _buffer.reserve(_options.data_page_size + 1024);
             _count = 0;
+            _raw_data_size = 0;
             _buffer.clear();
             _buffer.resize(PLAIN_PAGE_HEADER_SIZE);
         });
@@ -89,6 +91,8 @@ public:
 
     uint64_t size() const override { return _buffer.size(); }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         if (_count == 0) {
             return Status::Error<ErrorCode::ENTRY_NOT_FOUND>("page is empty");
@@ -111,6 +115,7 @@ private:
     faststring _buffer;
     PageBuilderOptions _options;
     size_t _count;
+    uint64_t _raw_data_size = 0;
     typedef typename TypeTraits<Type>::CppType CppType;
     enum { SIZE_OF_TYPE = TypeTraits<Type>::size };
     faststring _first_value;
diff --git a/be/src/olap/rowset/segment_v2/rle_page.h 
b/be/src/olap/rowset/segment_v2/rle_page.h
index 57e82ea8d8c..b5cc007e8ad 100644
--- a/be/src/olap/rowset/segment_v2/rle_page.h
+++ b/be/src/olap/rowset/segment_v2/rle_page.h
@@ -93,6 +93,7 @@ public:
         memcpy(&_last_value, &new_vals[*count - 1], SIZE_OF_TYPE);
 
         _count += *count;
+        _raw_data_size += *count * SIZE_OF_TYPE;
         return Status::OK();
     }
 
@@ -111,6 +112,7 @@ public:
         RETURN_IF_CATCH_EXCEPTION({
             _count = 0;
             _finished = false;
+            _raw_data_size = 0;
             _rle_encoder->Clear();
             _rle_encoder->Reserve(RLE_PAGE_HEADER_SIZE, 0);
         });
@@ -121,6 +123,8 @@ public:
 
     uint64_t size() const override { return _rle_encoder->len(); }
 
+    uint64_t get_raw_data_size() const override { return _raw_data_size; }
+
     Status get_first_value(void* value) const override {
         DCHECK(_finished);
         if (_count == 0) {
@@ -158,6 +162,7 @@ private:
     faststring _buf;
     CppType _first_value;
     CppType _last_value;
+    uint64_t _raw_data_size = 0;
 };
 
 template <FieldType Type>
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp 
b/be/src/olap/rowset/segment_v2/segment.cpp
index 0675c357ccb..096596a9979 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -791,6 +791,15 @@ Status Segment::get_column_reader(int32_t col_uid, 
std::shared_ptr<ColumnReader>
     return _column_reader_cache->get_column_reader(col_uid, column_reader, 
stats);
 }
 
+Status Segment::traverse_column_meta_pbs(const std::function<void(const 
ColumnMetaPB&)>& visitor) {
+    std::shared_ptr<SegmentFooterPB> footer_pb_shared;
+    RETURN_IF_ERROR(_get_segment_footer(footer_pb_shared, nullptr));
+    for (const auto& column : footer_pb_shared->columns()) {
+        visitor(column);
+    }
+    return Status::OK();
+}
+
 Status Segment::get_column_reader(const TabletColumn& col,
                                   std::shared_ptr<ColumnReader>* column_reader,
                                   OlapReaderStatistics* stats) {
diff --git a/be/src/olap/rowset/segment_v2/segment.h 
b/be/src/olap/rowset/segment_v2/segment.h
index b2e3e29ec6b..6a88e2ff731 100644
--- a/be/src/olap/rowset/segment_v2/segment.h
+++ b/be/src/olap/rowset/segment_v2/segment.h
@@ -210,6 +210,8 @@ public:
     Status get_column_reader(int32_t col_uid, std::shared_ptr<ColumnReader>* 
column_reader,
                              OlapReaderStatistics* stats);
 
+    Status traverse_column_meta_pbs(const std::function<void(const 
ColumnMetaPB&)>& visitor);
+
 private:
     DISALLOW_COPY_AND_ASSIGN(Segment);
     Segment(uint32_t segment_id, RowsetId rowset_id, TabletSchemaSPtr 
tablet_schema,
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp 
b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index a3a11ff93aa..0dbe2153778 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -1065,6 +1065,20 @@ void SegmentWriter::clear() {
 Status SegmentWriter::_write_data() {
     for (auto& column_writer : _column_writers) {
         RETURN_IF_ERROR(column_writer->write_data());
+
+        auto* column_meta = column_writer->get_column_meta();
+        DCHECK(column_meta != nullptr);
+        column_meta->set_compressed_data_bytes(
+                (column_meta->has_compressed_data_bytes() ? 
column_meta->compressed_data_bytes()
+                                                          : 0) +
+                column_writer->get_total_compressed_data_pages_bytes());
+        column_meta->set_uncompressed_data_bytes(
+                (column_meta->has_uncompressed_data_bytes() ? 
column_meta->uncompressed_data_bytes()
+                                                            : 0) +
+                column_writer->get_total_uncompressed_data_pages_bytes());
+        column_meta->set_raw_data_bytes(
+                (column_meta->has_raw_data_bytes() ? 
column_meta->raw_data_bytes() : 0) +
+                column_writer->get_raw_data_bytes());
     }
     return Status::OK();
 }
diff --git 
a/be/src/olap/rowset/segment_v2/variant/variant_column_writer_impl.cpp 
b/be/src/olap/rowset/segment_v2/variant/variant_column_writer_impl.cpp
index 354224f1d9a..0d39ec7de1c 100644
--- a/be/src/olap/rowset/segment_v2/variant/variant_column_writer_impl.cpp
+++ b/be/src/olap/rowset/segment_v2/variant/variant_column_writer_impl.cpp
@@ -544,7 +544,7 @@ Status VariantColumnWriterImpl::append_nullable(const 
uint8_t* null_map, const u
 VariantSubcolumnWriter::VariantSubcolumnWriter(const ColumnWriterOptions& opts,
                                                const TabletColumn* column,
                                                std::unique_ptr<Field> field)
-        : ColumnWriter(std::move(field), opts.meta->is_nullable()) {
+        : ColumnWriter(std::move(field), opts.meta->is_nullable(), opts.meta) {
     //
     _tablet_column = column;
     _opts = opts;
diff --git a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp 
b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
index faeab025760..66eaf5a9c4b 100644
--- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
@@ -932,6 +932,13 @@ Status VerticalSegmentWriter::write_batch() {
         for (auto& column_writer : _column_writers) {
             RETURN_IF_ERROR(column_writer->finish());
             RETURN_IF_ERROR(column_writer->write_data());
+
+            auto* column_meta = column_writer->get_column_meta();
+            column_meta->set_compressed_data_bytes(
+                    column_writer->get_total_compressed_data_pages_bytes());
+            column_meta->set_uncompressed_data_bytes(
+                    column_writer->get_total_uncompressed_data_pages_bytes());
+            
column_meta->set_raw_data_bytes(column_writer->get_raw_data_bytes());
         }
         return Status::OK();
     }
@@ -984,6 +991,13 @@ Status VerticalSegmentWriter::write_batch() {
         }
         RETURN_IF_ERROR(_column_writers[cid]->finish());
         RETURN_IF_ERROR(_column_writers[cid]->write_data());
+
+        auto* column_meta = _column_writers[cid]->get_column_meta();
+        column_meta->set_compressed_data_bytes(
+                _column_writers[cid]->get_total_compressed_data_pages_bytes());
+        column_meta->set_uncompressed_data_bytes(
+                
_column_writers[cid]->get_total_uncompressed_data_pages_bytes());
+        
column_meta->set_raw_data_bytes(_column_writers[cid]->get_raw_data_bytes());
     }
 
     for (auto& data : _batched_blocks) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java 
b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
index aafcae4a913..4adbf468c42 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
@@ -106,7 +106,9 @@ public enum SchemaTableType {
     SCH_CLUSTER_SNAPSHOT_PROPERTIES("CLUSTER_SNAPSHOT_PROPERTIES", 
"CLUSTER_SNAPSHOT_PROPERTIES",
             TSchemaTableType.SCH_CLUSTER_SNAPSHOT_PROPERTIES),
     SCH_BLACKHOLE("BLACKHOLE", "BLACKHOLE",
-            TSchemaTableType.SCH_BLACKHOLE);
+            TSchemaTableType.SCH_BLACKHOLE),
+    SCH_COLUMN_DATA_SIZES("COLUMN_DATA_SIZES", "COLUMN_DATA_SIZES",
+            TSchemaTableType.SCH_COLUMN_DATA_SIZES);
 
     private static final String dbName = "INFORMATION_SCHEMA";
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java 
b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
index ad24d006b4f..f3252cc15ec 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
@@ -634,6 +634,22 @@ public class SchemaTable extends Table {
                                     .column("REFRESH_INTERVAL_SECOND", 
ScalarType.createType(PrimitiveType.BIGINT))
                                     .build())
             )
+            .put("column_data_sizes",
+                    new SchemaTable(SystemIdGenerator.getNextId(), 
"column_data_sizes", TableType.SCHEMA,
+                            builder().column("BACKEND_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("TABLE_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("INDEX_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("PARTITION_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("TABLET_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("ROWSET_ID", 
ScalarType.createVarchar(64))
+                                    .column("COLUMN_UNIQUE_ID", 
ScalarType.createType(PrimitiveType.INT))
+                                    .column("COLUMN_NAME", 
ScalarType.createVarchar(64))
+                                    .column("COLUMN_TYPE", 
ScalarType.createVarchar(64))
+                                    .column("COMPRESSED_DATA_BYTES", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("UNCOMPRESSED_DATA_BYTES", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .column("RAW_DATA_BYTES", 
ScalarType.createType(PrimitiveType.BIGINT))
+                                    .build())
+            )
             .put("routine_load_jobs",
                     new SchemaTable(SystemIdGenerator.getNextId(), 
"routine_load_jobs", TableType.SCHEMA,
                             builder().column("JOB_ID", 
ScalarType.createStringType())
diff --git a/gensrc/proto/segment_v2.proto b/gensrc/proto/segment_v2.proto
index edd48248717..bff6642ea71 100644
--- a/gensrc/proto/segment_v2.proto
+++ b/gensrc/proto/segment_v2.proto
@@ -213,6 +213,10 @@ message ColumnMetaPB {
     optional VariantStatisticsPB variant_statistics = 21; // only used in 
variant type
     optional int32 variant_max_subcolumns_count = 22 [default = 0];
     optional uint64 none_null_size = 23 [default = 0];
+
+    optional uint64 compressed_data_bytes = 24;
+    optional uint64 uncompressed_data_bytes = 25;
+    optional uint64 raw_data_bytes = 26;
 }
 
 message PrimaryKeyIndexMetaPB {
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index a81a61a225a..f7c1e32daad 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -159,6 +159,7 @@ enum TSchemaTableType {
     SCH_CLUSTER_SNAPSHOTS = 60;
     SCH_CLUSTER_SNAPSHOT_PROPERTIES = 61;
     SCH_BLACKHOLE = 62;
+    SCH_COLUMN_DATA_SIZES = 63;
 }
 
 enum THdfsCompression {
diff --git 
a/regression-test/data/query_p0/system/test_query_sys_column_data_sizes.out 
b/regression-test/data/query_p0/system/test_query_sys_column_data_sizes.out
new file mode 100644
index 00000000000..8217f4cdbd5
--- /dev/null
+++ b/regression-test/data/query_p0/system/test_query_sys_column_data_sizes.out
@@ -0,0 +1,15 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !desc_column_data_sizes --
+BACKEND_ID     bigint  Yes     false   \N      
+TABLE_ID       bigint  Yes     false   \N      
+INDEX_ID       bigint  Yes     false   \N      
+PARTITION_ID   bigint  Yes     false   \N      
+TABLET_ID      bigint  Yes     false   \N      
+ROWSET_ID      varchar(64)     Yes     false   \N      
+COLUMN_UNIQUE_ID       int     Yes     false   \N      
+COLUMN_NAME    varchar(64)     Yes     false   \N      
+COLUMN_TYPE    varchar(64)     Yes     false   \N      
+COMPRESSED_DATA_BYTES  bigint  Yes     false   \N      
+UNCOMPRESSED_DATA_BYTES        bigint  Yes     false   \N      
+RAW_DATA_BYTES bigint  Yes     false   \N      
+
diff --git 
a/regression-test/suites/query_p0/system/test_query_sys_column_data_sizes.groovy
 
b/regression-test/suites/query_p0/system/test_query_sys_column_data_sizes.groovy
new file mode 100644
index 00000000000..2a07870f2cb
--- /dev/null
+++ 
b/regression-test/suites/query_p0/system/test_query_sys_column_data_sizes.groovy
@@ -0,0 +1,130 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+suite("test_query_sys_column_data_sizes", "query,p0") {
+    def tableName = "test_column_data_sizes_table"
+
+    qt_desc_column_data_sizes """ DESC information_schema.column_data_sizes """
+
+    sql "DROP TABLE IF EXISTS ${tableName};"
+    sql """
+        CREATE TABLE ${tableName} (
+            id INT NOT NULL,
+            name VARCHAR(100),
+            age TINYINT,
+            salary DECIMAL(10, 2),
+            is_active BOOLEAN,
+            birth_date DATE,
+            create_time DATETIME,
+            score DOUBLE,
+            description TEXT,
+            amount BIGINT,
+            tags ARRAY<VARCHAR(50)>,
+            scores_map MAP<VARCHAR(20), INT>,
+            person_info STRUCT<address:VARCHAR(200), phone:VARCHAR(20), 
email:VARCHAR(100)>
+        )
+        DUPLICATE KEY(id)
+        DISTRIBUTED BY HASH(id) BUCKETS 1
+        PROPERTIES (
+            "replication_num" = "1",
+            "disable_auto_compaction" = "true"
+        )
+    """
+
+    sql """
+        INSERT INTO ${tableName} VALUES
+        (1, 'Alice', 25, 50000.50, true, '1998-01-15', '2024-01-01 10:00:00', 
95.5, 'First employee', 1000000,
+         ['java', 'python', 'sql'], {'math':95, 'english':88}, 
named_struct('address', '123 Main St', 'phone', '1234567890', 'email', 
'[email protected]')),
+        (2, 'Bob', 30, 60000.75, true, '1993-05-20', '2024-01-02 11:00:00', 
88.0, 'Second employee', 2000000,
+         ['c++', 'go'], {'math':90, 'physics':85}, named_struct('address', 
'456 Oak Ave', 'phone', '0987654321', 'email', '[email protected]')),
+        (3, 'Charlie', 28, 55000.25, false, '1995-08-10', '2024-01-03 
12:00:00', 92.3, 'Third employee', 1500000,
+         ['javascript', 'typescript', 'react'], {'math':92, 'english':87, 
'history':90}, named_struct('address', '789 Pine Rd', 'phone', '5551234567', 
'email', '[email protected]'))
+    """
+
+    sql """
+        INSERT INTO ${tableName} VALUES
+        (4, 'David', 35, 70000.00, true, '1988-11-25', '2024-01-04 13:00:00', 
90.0, 'Fourth employee with a longer description text', 3000000,
+         ['rust', 'scala', 'haskell'], {'math':88, 'physics':92}, 
named_struct('address', '321 Elm St', 'phone', '1112223333', 'email', 
'[email protected]')),
+        (5, 'Eve', 27, 52000.50, false, '1996-03-12', '2024-01-05 14:00:00', 
87.5, 'Fifth employee', 1800000,
+         ['kotlin', 'swift'], {'math':85, 'chemistry':89}, 
named_struct('address', '654 Maple Dr', 'phone', '4445556666', 'email', 
'[email protected]'))
+    """
+
+    sql """
+        INSERT INTO ${tableName} VALUES
+        (6, 'Frank', 32, 65000.00, true, '1991-07-08', '2024-01-06 15:00:00', 
91.2, 'Sixth employee', 2500000,
+         ['ruby', 'perl', 'php'], {'math':91, 'english':89}, 
named_struct('address', '987 Cedar Ln', 'phone', '7778889999', 'email', 
'[email protected]')),
+        (7, 'Grace', 29, 58000.75, true, '1994-12-30', '2024-01-07 16:00:00', 
93.8, 'Seventh employee', 2200000,
+         ['r', 'matlab', 'julia'], {'math':94, 'statistics':96}, 
named_struct('address', '159 Birch Ct', 'phone', '2223334444', 'email', 
'[email protected]'))
+    """
+
+    sql "select * from ${tableName};"
+
+    // Get tablet ID for the test table
+    List<List<Object>> tablets = sql """ SHOW TABLETS FROM ${tableName} """
+    assert tablets.size() > 0, "Should have at least one tablet"
+    def tabletId = tablets[0][0]
+
+    def result_before = sql """
+        SELECT
+            COLUMN_NAME,
+            COLUMN_TYPE,
+            COUNT(DISTINCT ROWSET_ID) as rowset_count,
+            SUM(COMPRESSED_DATA_BYTES) as compressed_data_bytes,
+            SUM(UNCOMPRESSED_DATA_BYTES) as uncompressed_data_bytes,
+            SUM(RAW_DATA_BYTES) as raw_data_bytes
+        FROM information_schema.column_data_sizes
+        WHERE TABLET_ID = ${tabletId}
+        GROUP BY COLUMN_NAME, COLUMN_TYPE
+        ORDER BY COLUMN_NAME
+    """
+
+    logger.info("Column data sizes before compaction:")
+    for (row in result_before) {
+        logger.info("Column: ${row[0]}, Type: ${row[1]}, Rowset Count: 
${row[2]}, Compressed Size: ${row[3]}, Uncompressed Size: ${row[4]}, Raw Data 
Size: ${row[5]}")
+        assert row[3] > 0
+        assert row[4] > 0
+        assert row[5] > 0
+    }
+
+    assert result_before.size() == 13, "Should have 13 columns"
+
+    trigger_and_wait_compaction(tableName, "full")
+
+    def result_after = sql """
+        SELECT
+            COLUMN_NAME,
+            COLUMN_TYPE,
+            COUNT(DISTINCT ROWSET_ID) as rowset_count,
+            SUM(COMPRESSED_DATA_BYTES) as compressed_data_bytes,
+            SUM(UNCOMPRESSED_DATA_BYTES) as uncompressed_data_bytes,
+            SUM(RAW_DATA_BYTES) as raw_data_bytes
+        FROM information_schema.column_data_sizes
+        WHERE TABLET_ID = ${tabletId}
+        GROUP BY COLUMN_NAME, COLUMN_TYPE
+        ORDER BY COLUMN_NAME
+    """
+
+    logger.info("Column data sizes after compaction:")
+    for (row in result_after) {
+        logger.info("Column: ${row[0]}, Type: ${row[1]}, Rowset Count: 
${row[2]}, Compressed Size: ${row[3]}, Uncompressed Size: ${row[4]}, Raw Data 
Size: ${row[5]}")
+        assert row[3] > 0
+        assert row[4] > 0
+        assert row[5] > 0
+    }
+
+    assert result_after.size() == 13, "Should still have 13 columns after 
compaction"
+}
diff --git a/regression-test/suites/show_p0/test_show_data_warehouse.groovy 
b/regression-test/suites/show_p0/test_show_data_warehouse.groovy
index 628198a7ebd..eb9c98eaeac 100644
--- a/regression-test/suites/show_p0/test_show_data_warehouse.groovy
+++ b/regression-test/suites/show_p0/test_show_data_warehouse.groovy
@@ -91,12 +91,12 @@ suite("test_show_data_warehouse") {
         do {
             current = System.currentTimeMillis()
             result = sql """ show data 
properties("entire_warehouse"="true","db_names"="${db1Name}"); """
-            if ((result.size() == 2) && result[0][1].toInteger() == 921 * 
replicaCount1) {
+            if ((result.size() == 2) && result[0][1].toInteger() == 957 * 
replicaCount1) {
                 hitDb1 = true;
             }
 
             result = sql """ show data 
properties("entire_warehouse"="true","db_names"="${db2Name}"); """
-            if (result.size() == 2 && result[0][1].toInteger() == 898 * 
replicaCount1) {
+            if (result.size() == 2 && result[0][1].toInteger() == 934 * 
replicaCount1) {
                 hitDb2 = true;
             }
             if (hitDb1 && hitDb2) {
@@ -110,9 +110,9 @@ suite("test_show_data_warehouse") {
 
         result = sql """ show data 
properties("entire_warehouse"="true","db_names"="${db1Name},${db2Name}"); """
         assertEquals(result.size(), 3)
-        assertEquals(result[0][1].toInteger(), 921 * replicaCount1)
-        assertEquals(result[1][1].toInteger(), 898 * replicaCount1)
-        assertEquals(result[2][1].toInteger(), (921 + 898) * replicaCount1)
+        assertEquals(result[0][1].toInteger(), 957 * replicaCount1)
+        assertEquals(result[1][1].toInteger(), 934 * replicaCount1)
+        assertEquals(result[2][1].toInteger(), (957 + 934) * replicaCount1)
 
         result = sql """show data properties("entire_warehouse"="true")"""
         assertTrue(result.size() >= 3)


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

Reply via email to