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

zhangchen 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 18459b1eb19 [Opt](compile) Fix some compile warnings (#53684)
18459b1eb19 is described below

commit 18459b1eb1951da3342632ebec68522d4ed8ceef
Author: bobhan1 <[email protected]>
AuthorDate: Thu Jul 24 14:40:16 2025 +0800

    [Opt](compile) Fix some compile warnings (#53684)
    
    ### What problem does this PR solve?
    
    re-pull request https://github.com/apache/doris/pull/53318
---
 be/src/olap/base_tablet.cpp                        | 15 ++--
 be/src/olap/base_tablet.h                          |  2 +-
 be/src/olap/data_dir.cpp                           | 15 ++--
 be/src/olap/partial_update_info.cpp                | 79 +++++++++++-----------
 be/src/olap/partial_update_info.h                  | 38 +++++------
 be/src/olap/primary_key_index.h                    |  5 +-
 be/src/olap/rowid_conversion.h                     |  5 +-
 be/src/olap/rowset/beta_rowset.cpp                 | 12 ++--
 be/src/olap/rowset/beta_rowset_reader.cpp          |  4 +-
 be/src/olap/rowset/segment_v2/segment_writer.cpp   |  2 +-
 .../rowset/segment_v2/vertical_segment_writer.cpp  | 16 ++---
 11 files changed, 103 insertions(+), 90 deletions(-)

diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp
index 1ea9d8d860d..6706a820358 100644
--- a/be/src/olap/base_tablet.cpp
+++ b/be/src/olap/base_tablet.cpp
@@ -463,7 +463,7 @@ Status BaseTablet::lookup_row_data(const Slice& 
encoded_key, const RowLocation&
 Status BaseTablet::lookup_row_key(const Slice& encoded_key, TabletSchema* 
latest_schema,
                                   bool with_seq_col,
                                   const std::vector<RowsetSharedPtr>& 
specified_rowsets,
-                                  RowLocation* row_location, uint32_t version,
+                                  RowLocation* row_location, int64_t version,
                                   
std::vector<std::unique_ptr<SegmentCacheHandle>>& segment_caches,
                                   RowsetSharedPtr* rowset, bool with_rowid,
                                   std::string* encoded_seq_value, 
OlapReaderStatistics* stats,
@@ -639,9 +639,9 @@ Status 
BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset,
 
     RETURN_IF_ERROR(seg->load_pk_index_and_bf(nullptr)); // We need index 
blocks to iterate
     const auto* pk_idx = seg->get_primary_key_index();
-    int total = pk_idx->num_rows();
+    int64_t total = pk_idx->num_rows();
     uint32_t row_id = 0;
-    int32_t remaining = total;
+    int64_t remaining = total;
     bool exact_match = false;
     std::string last_key;
     int batch_size = 1024;
@@ -653,7 +653,7 @@ Status 
BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset,
         std::unique_ptr<segment_v2::IndexedColumnIterator> iter;
         RETURN_IF_ERROR(pk_idx->new_iterator(&iter, nullptr));
 
-        size_t num_to_read = std::min(batch_size, remaining);
+        size_t num_to_read = std::min<int64_t>(batch_size, remaining);
         auto index_type = 
vectorized::DataTypeFactory::instance().create_data_type(
                 pk_idx->type_info()->type(), 1, 0);
         auto index_column = index_type->create_column();
@@ -721,12 +721,11 @@ Status 
BaseTablet::calc_segment_delete_bitmap(RowsetSharedPtr rowset,
             Status st = Status::OK();
             if (tablet_delete_bitmap == nullptr) {
                 st = lookup_row_key(key, rowset_schema.get(), true, 
specified_rowsets, &loc,
-                                    cast_set<uint32_t>(dummy_version.first - 
1), segment_caches,
-                                    &rowset_find);
+                                    dummy_version.first - 1, segment_caches, 
&rowset_find);
             } else {
                 st = lookup_row_key(key, rowset_schema.get(), true, 
specified_rowsets, &loc,
-                                    cast_set<uint32_t>(dummy_version.first - 
1), segment_caches,
-                                    &rowset_find, true, nullptr, nullptr, 
tablet_delete_bitmap);
+                                    dummy_version.first - 1, segment_caches, 
&rowset_find, true,
+                                    nullptr, nullptr, tablet_delete_bitmap);
             }
             bool expected_st = st.ok() || st.is<KEY_NOT_FOUND>() || 
st.is<KEY_ALREADY_EXISTS>();
             // It's a defensive DCHECK, we need to exclude some common errors 
to avoid core-dump
diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h
index 8a4f54a25bf..53b821ae379 100644
--- a/be/src/olap/base_tablet.h
+++ b/be/src/olap/base_tablet.h
@@ -163,7 +163,7 @@ public:
     //       not supported error in other data model.
     Status lookup_row_key(const Slice& encoded_key, TabletSchema* 
latest_schema, bool with_seq_col,
                           const std::vector<RowsetSharedPtr>& 
specified_rowsets,
-                          RowLocation* row_location, uint32_t version,
+                          RowLocation* row_location, int64_t version,
                           std::vector<std::unique_ptr<SegmentCacheHandle>>& 
segment_caches,
                           RowsetSharedPtr* rowset = nullptr, bool with_rowid = 
true,
                           std::string* encoded_seq_value = nullptr,
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index 40dc21fb4dc..c5e7edcf6f3 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -38,6 +38,7 @@
 #include <thread>
 #include <utility>
 
+#include "common/cast_set.h"
 #include "common/config.h"
 #include "common/logging.h"
 #include "io/fs/file_reader.h"
@@ -67,6 +68,7 @@
 #include "util/uid_util.h"
 
 namespace doris {
+#include "common/compile_check_begin.h"
 using namespace ErrorCode;
 
 namespace {
@@ -382,7 +384,7 @@ Status DataDir::load() {
                         rowset_id.to_string(), tablet_uid.to_string());
                 CHECK_EQ(orig_delete_sub_pred.size(), 
delete_pred->sub_predicates().size())
                         << "inconsistent sub predicate v1 after conversion";
-                for (size_t i = 0; i < orig_delete_sub_pred.size(); ++i) {
+                for (int i = 0; i < orig_delete_sub_pred.size(); ++i) {
                     CHECK_STREQ(orig_delete_sub_pred.Get(i).c_str(),
                                 delete_pred->sub_predicates().Get(i).c_str())
                             << "inconsistent sub predicate v1 after 
conversion";
@@ -484,7 +486,8 @@ Status DataDir::load() {
     auto load_pending_publish_info_func =
             [&engine = _engine](int64_t tablet_id, int64_t publish_version, 
std::string_view info) {
                 PendingPublishInfoPB pending_publish_info_pb;
-                bool parsed = 
pending_publish_info_pb.ParseFromArray(info.data(), info.size());
+                bool parsed = 
pending_publish_info_pb.ParseFromArray(info.data(),
+                                                                     
cast_set<int>(info.size()));
                 if (!parsed) {
                     LOG(WARNING) << "parse pending publish info failed, 
tablet_id: " << tablet_id
                                  << " publish_version: " << publish_version;
@@ -619,13 +622,13 @@ Status DataDir::load() {
         }
 
         DeleteBitmapPB delete_bitmap_pb;
-        delete_bitmap_pb.ParseFromArray(val.data(), val.size());
+        delete_bitmap_pb.ParseFromArray(val.data(), cast_set<int>(val.size()));
         int rst_ids_size = delete_bitmap_pb.rowset_ids_size();
         int seg_ids_size = delete_bitmap_pb.segment_ids_size();
         int seg_maps_size = delete_bitmap_pb.segment_delete_bitmaps_size();
         CHECK(rst_ids_size == seg_ids_size && seg_ids_size == seg_maps_size);
 
-        for (size_t i = 0; i < rst_ids_size; ++i) {
+        for (int i = 0; i < rst_ids_size; ++i) {
             RowsetId rst_id;
             rst_id.init(delete_bitmap_pb.rowset_ids(i));
             // only process the rowset in _rs_metas
@@ -869,7 +872,7 @@ void DataDir::perform_path_gc() {
                 }
                 int16_t shard_id = -1;
                 try {
-                    shard_id = std::stoi(shard.file_name);
+                    shard_id = cast_set<int16_t>(std::stoi(shard.file_name));
                 } catch (const std::exception&) {
                     LOG(WARNING) << "failed to stoi shard_id, shard name=" << 
shard.file_name;
                     continue;
@@ -1092,5 +1095,5 @@ void DataDir::perform_remote_tablet_gc() {
         static_cast<void>(_meta->remove(META_COLUMN_FAMILY_INDEX, key));
     }
 }
-
+#include "common/compile_check_end.h"
 } // namespace doris
diff --git a/be/src/olap/partial_update_info.cpp 
b/be/src/olap/partial_update_info.cpp
index b5ce600fe28..f4dd4529713 100644
--- a/be/src/olap/partial_update_info.cpp
+++ b/be/src/olap/partial_update_info.cpp
@@ -19,6 +19,8 @@
 
 #include <gen_cpp/olap_file.pb.h>
 
+#include <cstdint>
+
 #include "common/consts.h"
 #include "common/logging.h"
 #include "olap/base_tablet.h"
@@ -36,13 +38,13 @@
 #include "vec/olap/olap_data_convertor.h"
 
 namespace doris {
-
+#include "common/compile_check_begin.h"
 Status PartialUpdateInfo::init(int64_t tablet_id, int64_t txn_id, const 
TabletSchema& tablet_schema,
                                UniqueKeyUpdateModePB unique_key_update_mode,
                                PartialUpdateNewRowPolicyPB policy,
                                const std::set<std::string>& 
partial_update_cols,
-                               bool is_strict_mode, int64_t timestamp_ms, 
int32_t nano_seconds,
-                               const std::string& timezone,
+                               bool is_strict_mode_, int64_t timestamp_ms_, 
int32_t nano_seconds_,
+                               const std::string& timezone_,
                                const std::string& auto_increment_column,
                                int32_t sequence_map_col_uid, int64_t 
cur_max_version) {
     partial_update_mode = unique_key_update_mode;
@@ -50,9 +52,9 @@ Status PartialUpdateInfo::init(int64_t tablet_id, int64_t 
txn_id, const TabletSc
     partial_update_input_columns = partial_update_cols;
     max_version_in_flush_phase = cur_max_version;
     sequence_map_col_unqiue_id = sequence_map_col_uid;
-    this->timestamp_ms = timestamp_ms;
-    this->nano_seconds = nano_seconds;
-    this->timezone = timezone;
+    timestamp_ms = timestamp_ms_;
+    nano_seconds = nano_seconds_;
+    timezone = timezone_;
     missing_cids.clear();
     update_cids.clear();
 
@@ -93,7 +95,7 @@ Status PartialUpdateInfo::init(int64_t tablet_id, int64_t 
txn_id, const TabletSc
             }
         }
     }
-    this->is_strict_mode = is_strict_mode;
+    is_strict_mode = is_strict_mode_;
     is_input_columns_contains_auto_inc_column =
             is_fixed_partial_update() &&
             partial_update_input_columns.contains(auto_increment_column);
@@ -319,7 +321,7 @@ Status FixedReadPlan::read_columns_by_plan(
     }
     bool has_row_column = tablet_schema.has_row_store_for_all_columns();
     auto mutable_columns = block.mutate_columns();
-    size_t read_idx = 0;
+    uint32_t read_idx = 0;
     for (const auto& [rowset_id, segment_row_mappings] : plan) {
         for (const auto& [segment_id, mappings] : segment_row_mappings) {
             auto rowset_iter = rsid_to_rowset.find(rowset_id);
@@ -330,7 +332,7 @@ Status FixedReadPlan::read_columns_by_plan(
                     continue;
                 }
                 rids.emplace_back(rid);
-                (*read_index)[pos] = read_idx++;
+                (*read_index)[static_cast<uint32_t>(pos)] = read_idx++;
             }
             if (has_row_column) {
                 auto st = BaseTablet::fetch_value_through_row_column(
@@ -361,7 +363,7 @@ Status FixedReadPlan::fill_missing_columns(
         RowsetWriterContext* rowset_ctx, const std::map<RowsetId, 
RowsetSharedPtr>& rsid_to_rowset,
         const TabletSchema& tablet_schema, vectorized::Block& full_block,
         const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-        const size_t& segment_start_pos, const vectorized::Block* block) const 
{
+        uint32_t segment_start_pos, const vectorized::Block* block) const {
     auto mutable_full_columns = full_block.mutate_columns();
     // create old value columns
     const auto& missing_cids = rowset_ctx->partial_update_info->missing_cids;
@@ -453,8 +455,8 @@ void FlexibleReadPlan::prepare_to_read(const RowLocation& 
row_location, size_t p
                                        const BitmapValue& skip_bitmap) {
     if (!use_row_store) {
         for (uint64_t col_uid : skip_bitmap) {
-            
plan[row_location.rowset_id][row_location.segment_id][col_uid].emplace_back(
-                    row_location.row_id, pos);
+            
plan[row_location.rowset_id][row_location.segment_id][static_cast<uint32_t>(col_uid)]
+                    .emplace_back(row_location.row_id, pos);
         }
     } else {
         
row_store_plan[row_location.rowset_id][row_location.segment_id].emplace_back(
@@ -471,7 +473,7 @@ Status FlexibleReadPlan::read_columns_by_plan(
 
     // cid -> next rid to fill in block
     std::map<uint32_t, uint32_t> next_read_idx;
-    for (std::size_t cid {0}; cid < tablet_schema.num_columns(); cid++) {
+    for (uint32_t cid {0}; cid < tablet_schema.num_columns(); cid++) {
         next_read_idx[cid] = 0;
     }
 
@@ -486,7 +488,7 @@ Status FlexibleReadPlan::read_columns_by_plan(
                 std::vector<uint32_t> rids;
                 for (auto [rid, pos] : mappings) {
                     rids.emplace_back(rid);
-                    (*read_index)[cid][pos] = next_read_idx[cid]++;
+                    (*read_index)[cid][static_cast<uint32_t>(pos)] = 
next_read_idx[cid]++;
                 }
 
                 TabletColumn tablet_column = tablet_schema.column(cid);
@@ -507,7 +509,7 @@ Status FlexibleReadPlan::read_columns_by_plan(
         const std::map<RowsetId, RowsetSharedPtr>& rsid_to_rowset,
         vectorized::Block& old_value_block, std::map<uint32_t, uint32_t>* 
read_index) const {
     DCHECK(use_row_store);
-    size_t read_idx = 0;
+    uint32_t read_idx = 0;
     for (const auto& [rowset_id, segment_row_mappings] : row_store_plan) {
         for (const auto& [segment_id, mappings] : segment_row_mappings) {
             auto rowset_iter = rsid_to_rowset.find(rowset_id);
@@ -515,7 +517,7 @@ Status FlexibleReadPlan::read_columns_by_plan(
             std::vector<uint32_t> rids;
             for (auto [rid, pos] : mappings) {
                 rids.emplace_back(rid);
-                (*read_index)[pos] = read_idx++;
+                (*read_index)[static_cast<uint32_t>(pos)] = read_idx++;
             }
             auto st = 
BaseTablet::fetch_value_through_row_column(rowset_iter->second, tablet_schema,
                                                                  segment_id, 
rids, cids_to_read,
@@ -533,8 +535,8 @@ Status FlexibleReadPlan::fill_non_primary_key_columns(
         RowsetWriterContext* rowset_ctx, const std::map<RowsetId, 
RowsetSharedPtr>& rsid_to_rowset,
         const TabletSchema& tablet_schema, vectorized::Block& full_block,
         const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-        const std::size_t segment_start_pos, const std::size_t block_start_pos,
-        const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const {
+        uint32_t segment_start_pos, uint32_t block_start_pos, const 
vectorized::Block* block,
+        std::vector<BitmapValue>* skip_bitmaps) const {
     auto mutable_full_columns = full_block.mutate_columns();
 
     // missing_cids are all non sort key columns' cids
@@ -562,8 +564,8 @@ Status 
FlexibleReadPlan::fill_non_primary_key_columns_for_column_store(
         const TabletSchema& tablet_schema, const std::vector<uint32_t>& 
non_sort_key_cids,
         vectorized::Block& old_value_block, vectorized::MutableColumns& 
mutable_full_columns,
         const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-        const std::size_t segment_start_pos, const std::size_t block_start_pos,
-        const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const {
+        uint32_t segment_start_pos, uint32_t block_start_pos, const 
vectorized::Block* block,
+        std::vector<BitmapValue>* skip_bitmaps) const {
     auto* info = rowset_ctx->partial_update_info.get();
     int32_t seq_col_unique_id = -1;
     if (tablet_schema.has_sequence_col()) {
@@ -590,7 +592,7 @@ Status 
FlexibleReadPlan::fill_non_primary_key_columns_for_column_store(
                                  const vectorized::IColumn& default_value_col,
                                  const vectorized::IColumn& old_value_col,
                                  const vectorized::IColumn& cur_col, 
std::size_t block_pos,
-                                 std::size_t segment_pos, bool skipped, bool 
row_has_sequence_col,
+                                 uint32_t segment_pos, bool skipped, bool 
row_has_sequence_col,
                                  bool use_default, const signed char* 
delete_sign_column_data) {
         if (skipped) {
             DCHECK(cid != tablet_schema.skip_bitmap_col_idx());
@@ -675,8 +677,8 @@ Status 
FlexibleReadPlan::fill_non_primary_key_columns_for_row_store(
         const TabletSchema& tablet_schema, const std::vector<uint32_t>& 
non_sort_key_cids,
         vectorized::Block& old_value_block, vectorized::MutableColumns& 
mutable_full_columns,
         const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-        const std::size_t segment_start_pos, const std::size_t block_start_pos,
-        const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const {
+        uint32_t segment_start_pos, uint32_t block_start_pos, const 
vectorized::Block* block,
+        std::vector<BitmapValue>* skip_bitmaps) const {
     auto* info = rowset_ctx->partial_update_info.get();
     int32_t seq_col_unique_id = -1;
     if (tablet_schema.has_sequence_col()) {
@@ -782,7 +784,7 @@ 
BlockAggregator::BlockAggregator(segment_v2::VerticalSegmentWriter& vertical_seg
         : _writer(vertical_segment_writer), 
_tablet_schema(*_writer._tablet_schema) {}
 
 void BlockAggregator::merge_one_row(vectorized::MutableBlock& dst_block,
-                                    vectorized::Block* src_block, int64_t rid,
+                                    vectorized::Block* src_block, int rid,
                                     BitmapValue& skip_bitmap) {
     for (size_t cid {_tablet_schema.num_key_columns()}; cid < 
_tablet_schema.num_columns(); cid++) {
         if (cid == _tablet_schema.skip_bitmap_col_idx()) {
@@ -808,7 +810,7 @@ void 
BlockAggregator::merge_one_row(vectorized::MutableBlock& dst_block,
 }
 
 void BlockAggregator::append_one_row(vectorized::MutableBlock& dst_block,
-                                     vectorized::Block* src_block, int64_t 
rid) {
+                                     vectorized::Block* src_block, int rid) {
     dst_block.add_row(src_block, rid);
     _state.rows++;
     VLOG_DEBUG << fmt::format("append a new row, after append, 
output_block.rows()={}, state: {}",
@@ -825,7 +827,7 @@ void 
BlockAggregator::remove_last_n_rows(vectorized::MutableBlock& dst_block, in
 }
 
 void BlockAggregator::append_or_merge_row(vectorized::MutableBlock& dst_block,
-                                          vectorized::Block* src_block, 
int64_t rid,
+                                          vectorized::Block* src_block, int 
rid,
                                           BitmapValue& skip_bitmap, bool 
have_delete_sign) {
     if (have_delete_sign) {
         // remove all the previous batched rows
@@ -844,9 +846,9 @@ void 
BlockAggregator::append_or_merge_row(vectorized::MutableBlock& dst_block,
 };
 
 Status BlockAggregator::aggregate_rows(
-        vectorized::MutableBlock& output_block, vectorized::Block* block, 
int64_t start,
-        int64_t end, std::string key, std::vector<BitmapValue>* skip_bitmaps,
-        const signed char* delete_signs, vectorized::IOlapColumnDataAccessor* 
seq_column,
+        vectorized::MutableBlock& output_block, vectorized::Block* block, int 
start, int end,
+        std::string key, std::vector<BitmapValue>* skip_bitmaps, const signed 
char* delete_signs,
+        vectorized::IOlapColumnDataAccessor* seq_column,
         const std::vector<RowsetSharedPtr>& specified_rowsets,
         std::vector<std::unique_ptr<SegmentCacheHandle>>& segment_caches) {
     VLOG_DEBUG << fmt::format("merge rows in range=[{}-{})", start, end);
@@ -868,7 +870,7 @@ Status BlockAggregator::aggregate_rows(
     Status st = _writer._tablet->lookup_row_key(
             key, &_tablet_schema, false, specified_rowsets, &loc, 
_writer._mow_context->max_version,
             segment_caches, &rowset, true, &previous_encoded_seq_value);
-    int64_t pos = start;
+    int pos = start;
     bool is_expected_st = (st.is<ErrorCode::KEY_NOT_FOUND>() || st.ok());
     DCHECK(is_expected_st || st.is<ErrorCode::MEM_LIMIT_EXCEEDED>())
             << "[BlockAggregator::aggregate_rows] unexpected error status 
while lookup_row_key:"
@@ -911,7 +913,7 @@ Status BlockAggregator::aggregate_rows(
         }
     }
 
-    for (int64_t rid {pos}; rid < end; rid++) {
+    for (int rid {pos}; rid < end; rid++) {
         auto& skip_bitmap = skip_bitmaps->at(rid);
         bool row_has_sequence_col = (!skip_bitmap.contains(seq_col_unique_id));
         bool have_delete_sign =
@@ -934,7 +936,7 @@ Status BlockAggregator::aggregate_rows(
 };
 
 Status BlockAggregator::aggregate_for_sequence_column(
-        vectorized::Block* block, size_t num_rows,
+        vectorized::Block* block, int num_rows,
         const std::vector<vectorized::IOlapColumnDataAccessor*>& key_columns,
         vectorized::IOlapColumnDataAccessor* seq_column,
         const std::vector<RowsetSharedPtr>& specified_rowsets,
@@ -956,7 +958,7 @@ Status BlockAggregator::aggregate_for_sequence_column(
 
     int same_key_rows {0};
     std::string previous_key {};
-    for (size_t block_pos {0}; block_pos < num_rows; block_pos++) {
+    for (int block_pos {0}; block_pos < num_rows; block_pos++) {
         std::string key = _writer._full_encode_keys(key_columns, block_pos);
         if (block_pos > 0 && previous_key == key) {
             same_key_rows++;
@@ -997,7 +999,7 @@ Status 
BlockAggregator::fill_sequence_column(vectorized::Block* block, size_t nu
     auto new_seq_col_ptr = 
tmp_block.get_by_position(0).column->assume_mutable();
     const auto& old_seq_col_ptr = *seq_col_block.get_by_position(0).column;
     const auto& cur_seq_col_ptr = 
*block->get_by_position(_tablet_schema.sequence_col_idx()).column;
-    for (size_t block_pos {0}; block_pos < num_rows; block_pos++) {
+    for (uint32_t block_pos {0}; block_pos < num_rows; block_pos++) {
         if (read_index.contains(block_pos)) {
             new_seq_col_ptr->insert_from(old_seq_col_ptr, 
read_index[block_pos]);
             skip_bitmaps[block_pos].remove(seq_col_unique_id);
@@ -1109,7 +1111,7 @@ Status BlockAggregator::filter_block(vectorized::Block* 
block, size_t num_rows,
             {std::move(filter_column), 
std::make_shared<vectorized::DataTypeUInt8>(), col_name});
     RETURN_IF_ERROR(vectorized::Block::filter_block(block, num_cols, 
num_cols));
     DCHECK_EQ(num_cols, block->columns());
-    int merged_rows = num_rows - block->rows();
+    size_t merged_rows = num_rows - block->rows();
     if (duplicate_rows != merged_rows) {
         auto msg = fmt::format(
                 "filter_block_for_flexible_partial_update {}: duplicate_rows 
!= merged_rows, "
@@ -1125,7 +1127,7 @@ Status BlockAggregator::convert_pk_columns(
         vectorized::Block* block, size_t row_pos, size_t num_rows,
         std::vector<vectorized::IOlapColumnDataAccessor*>& key_columns) {
     key_columns.clear();
-    for (std::size_t cid {0}; cid < _tablet_schema.num_key_columns(); cid++) {
+    for (uint32_t cid {0}; cid < _tablet_schema.num_key_columns(); cid++) {
         
RETURN_IF_ERROR(_writer._olap_data_convertor->set_source_content_with_specifid_column(
                 block->get_by_position(cid), row_pos, num_rows, cid));
         auto [status, column] = 
_writer._olap_data_convertor->convert_column_data(cid);
@@ -1169,8 +1171,9 @@ Status 
BlockAggregator::aggregate_for_flexible_partial_update(
     // some of them don't. We can't do the de-duplication in memtable because 
we don't know the historical data. We must
     // de-duplicate them here.
     if (_tablet_schema.has_sequence_col()) {
-        RETURN_IF_ERROR(aggregate_for_sequence_column(block, num_rows, 
key_columns, seq_column,
-                                                      specified_rowsets, 
segment_caches));
+        RETURN_IF_ERROR(aggregate_for_sequence_column(block, 
static_cast<int>(num_rows),
+                                                      key_columns, seq_column, 
specified_rowsets,
+                                                      segment_caches));
     }
 
     // 2. merge duplicate rows and handle insert after delete
diff --git a/be/src/olap/partial_update_info.h 
b/be/src/olap/partial_update_info.h
index fbf3dad71ad..44bc3b47bce 100644
--- a/be/src/olap/partial_update_info.h
+++ b/be/src/olap/partial_update_info.h
@@ -132,7 +132,7 @@ public:
                                 const std::map<RowsetId, RowsetSharedPtr>& 
rsid_to_rowset,
                                 const TabletSchema& tablet_schema, 
vectorized::Block& full_block,
                                 const std::vector<bool>& 
use_default_or_null_flag,
-                                bool has_default_or_nullable, const size_t& 
segment_start_pos,
+                                bool has_default_or_nullable, uint32_t 
segment_start_pos,
                                 const vectorized::Block* block) const;
 
 private:
@@ -156,13 +156,14 @@ public:
                                 const std::map<RowsetId, RowsetSharedPtr>& 
rsid_to_rowset,
                                 vectorized::Block& old_value_block,
                                 std::map<uint32_t, uint32_t>* read_index) 
const;
-    Status fill_non_primary_key_columns(
-            RowsetWriterContext* rowset_ctx,
-            const std::map<RowsetId, RowsetSharedPtr>& rsid_to_rowset,
-            const TabletSchema& tablet_schema, vectorized::Block& full_block,
-            const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-            const std::size_t segment_start_pos, const std::size_t 
block_start_pos,
-            const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const;
+    Status fill_non_primary_key_columns(RowsetWriterContext* rowset_ctx,
+                                        const std::map<RowsetId, 
RowsetSharedPtr>& rsid_to_rowset,
+                                        const TabletSchema& tablet_schema,
+                                        vectorized::Block& full_block,
+                                        const std::vector<bool>& 
use_default_or_null_flag,
+                                        bool has_default_or_nullable, uint32_t 
segment_start_pos,
+                                        uint32_t block_start_pos, const 
vectorized::Block* block,
+                                        std::vector<BitmapValue>* 
skip_bitmaps) const;
 
     Status fill_non_primary_key_columns_for_column_store(
             RowsetWriterContext* rowset_ctx,
@@ -170,16 +171,16 @@ public:
             const TabletSchema& tablet_schema, const std::vector<uint32_t>& 
non_sort_key_cids,
             vectorized::Block& old_value_block, vectorized::MutableColumns& 
mutable_full_columns,
             const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-            const std::size_t segment_start_pos, const std::size_t 
block_start_pos,
-            const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const;
+            uint32_t segment_start_pos, uint32_t block_start_pos, const 
vectorized::Block* block,
+            std::vector<BitmapValue>* skip_bitmaps) const;
     Status fill_non_primary_key_columns_for_row_store(
             RowsetWriterContext* rowset_ctx,
             const std::map<RowsetId, RowsetSharedPtr>& rsid_to_rowset,
             const TabletSchema& tablet_schema, const std::vector<uint32_t>& 
non_sort_key_cids,
             vectorized::Block& old_value_block, vectorized::MutableColumns& 
mutable_full_columns,
             const std::vector<bool>& use_default_or_null_flag, bool 
has_default_or_nullable,
-            const std::size_t segment_start_pos, const std::size_t 
block_start_pos,
-            const vectorized::Block* block, std::vector<BitmapValue>* 
skip_bitmaps) const;
+            uint32_t segment_start_pos, uint32_t block_start_pos, const 
vectorized::Block* block,
+            std::vector<BitmapValue>* skip_bitmaps) const;
 
 private:
     bool use_row_store {false};
@@ -204,7 +205,7 @@ public:
 
 private:
     Status aggregate_for_sequence_column(
-            vectorized::Block* block, size_t num_rows,
+            vectorized::Block* block, int num_rows,
             const std::vector<vectorized::IOlapColumnDataAccessor*>& 
key_columns,
             vectorized::IOlapColumnDataAccessor* seq_column,
             const std::vector<RowsetSharedPtr>& specified_rowsets,
@@ -223,16 +224,15 @@ private:
                                 std::vector<BitmapValue>& skip_bitmaps);
 
     void append_or_merge_row(vectorized::MutableBlock& dst_block, 
vectorized::Block* src_block,
-                             int64_t rid, BitmapValue& skip_bitmap, bool 
have_delete_sign);
-    void merge_one_row(vectorized::MutableBlock& dst_block, vectorized::Block* 
src_block,
-                       int64_t rid, BitmapValue& skip_bitmap);
-    void append_one_row(vectorized::MutableBlock& dst_block, 
vectorized::Block* src_block,
-                        int64_t rid);
+                             int rid, BitmapValue& skip_bitmap, bool 
have_delete_sign);
+    void merge_one_row(vectorized::MutableBlock& dst_block, vectorized::Block* 
src_block, int rid,
+                       BitmapValue& skip_bitmap);
+    void append_one_row(vectorized::MutableBlock& dst_block, 
vectorized::Block* src_block, int rid);
     void remove_last_n_rows(vectorized::MutableBlock& dst_block, int n);
 
     // aggregate rows with same keys in range [start, end) from block to 
output_block
     Status aggregate_rows(vectorized::MutableBlock& output_block, 
vectorized::Block* block,
-                          int64_t start, int64_t end, std::string key,
+                          int start, int end, std::string key,
                           std::vector<BitmapValue>* skip_bitmaps, const signed 
char* delete_signs,
                           vectorized::IOlapColumnDataAccessor* seq_column,
                           const std::vector<RowsetSharedPtr>& 
specified_rowsets,
diff --git a/be/src/olap/primary_key_index.h b/be/src/olap/primary_key_index.h
index 0f41fde72c0..75920eb37be 100644
--- a/be/src/olap/primary_key_index.h
+++ b/be/src/olap/primary_key_index.h
@@ -34,6 +34,7 @@
 #include "util/slice.h"
 
 namespace doris {
+#include "common/compile_check_begin.h"
 class TypeInfo;
 
 namespace io {
@@ -132,7 +133,7 @@ public:
         return _bf->test_bytes(key.data, key.size);
     }
 
-    uint32_t num_rows() const {
+    int64_t num_rows() const {
         DCHECK(_index_parsed);
         return _index_reader->num_values();
     }
@@ -157,5 +158,5 @@ private:
     size_t _bf_num = 0;
     uint64_t _bf_bytes = 0;
 };
-
+#include "common/compile_check_end.h"
 } // namespace doris
diff --git a/be/src/olap/rowid_conversion.h b/be/src/olap/rowid_conversion.h
index f6a6a263980..7dc66924f20 100644
--- a/be/src/olap/rowid_conversion.h
+++ b/be/src/olap/rowid_conversion.h
@@ -20,11 +20,13 @@
 #include <map>
 #include <vector>
 
+#include "common/cast_set.h"
 #include "olap/olap_common.h"
 #include "olap/utils.h"
 #include "runtime/thread_context.h"
 
 namespace doris {
+#include "common/compile_check_begin.h"
 
 // For unique key merge on write table, we should update delete bitmap
 // of destination rowset when compaction finished.
@@ -58,7 +60,7 @@ public:
                                 ->consumption()));
             }
 
-            uint32_t id = _segments_rowid_map.size();
+            uint32_t id = static_cast<uint32_t>(_segments_rowid_map.size());
             _segment_to_id_map.emplace(std::pair<RowsetId, uint32_t> 
{src_rowset_id, i}, id);
             _id_to_segment_map.emplace_back(src_rowset_id, i);
             std::vector<std::pair<uint32_t, uint32_t>> vec(
@@ -172,4 +174,5 @@ private:
     std::uint32_t _cur_dst_segment_rowid = 0;
 };
 
+#include "common/compile_check_end.h"
 } // namespace doris
diff --git a/be/src/olap/rowset/beta_rowset.cpp 
b/be/src/olap/rowset/beta_rowset.cpp
index 86a9877b5ed..125105bca32 100644
--- a/be/src/olap/rowset/beta_rowset.cpp
+++ b/be/src/olap/rowset/beta_rowset.cpp
@@ -50,6 +50,7 @@
 #include "util/doris_metrics.h"
 
 namespace doris {
+#include "common/compile_check_begin.h"
 using namespace ErrorCode;
 
 std::string BetaRowset::local_segment_path_segcompacted(const std::string& 
tablet_path,
@@ -192,12 +193,13 @@ Status BetaRowset::load_segment(int64_t seg_id, 
OlapReaderStatistics* stats,
                                                     : 
io::FileCachePolicy::NO_CACHE,
             .is_doris_table = true,
             .cache_base_path = "",
-            .file_size = _rowset_meta->segment_file_size(seg_id),
+            .file_size = 
_rowset_meta->segment_file_size(static_cast<int>(seg_id)),
     };
 
-    auto s = segment_v2::Segment::open(fs, seg_path, 
_rowset_meta->tablet_id(), seg_id, rowset_id(),
-                                       _schema, reader_options, segment,
-                                       
_rowset_meta->inverted_index_file_info(seg_id), stats);
+    auto s = segment_v2::Segment::open(
+            fs, seg_path, _rowset_meta->tablet_id(), 
static_cast<uint32_t>(seg_id), rowset_id(),
+            _schema, reader_options, segment,
+            _rowset_meta->inverted_index_file_info(static_cast<int>(seg_id)), 
stats);
     if (!s.ok()) {
         LOG(WARNING) << "failed to open segment. " << seg_path << " under 
rowset " << rowset_id()
                      << " : " << s.to_string();
@@ -868,5 +870,5 @@ Status BetaRowset::show_nested_index_file(rapidjson::Value* 
rowset_value,
     rowset_value->AddMember("segments", segments, allocator);
     return Status::OK();
 }
-
+#include "common/compile_check_end.h"
 } // namespace doris
diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp 
b/be/src/olap/rowset/beta_rowset_reader.cpp
index 1e1cb417aa2..8f46e41bfc7 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -49,6 +49,7 @@
 #include "vec/olap/vgeneric_iterators.h"
 
 namespace doris {
+#include "common/compile_check_begin.h"
 using namespace ErrorCode;
 
 BetaRowsetReader::BetaRowsetReader(BetaRowsetSharedPtr rowset)
@@ -297,7 +298,7 @@ Status BetaRowsetReader::_init_iterator() {
     if (_is_merge_iterator()) {
         auto sequence_loc = -1;
         if (_read_context->sequence_id_idx != -1) {
-            for (size_t loc = 0; loc < _read_context->return_columns->size(); 
loc++) {
+            for (int loc = 0; loc < _read_context->return_columns->size(); 
loc++) {
                 if (_read_context->return_columns->at(loc) == 
_read_context->sequence_id_idx) {
                     sequence_loc = loc;
                     break;
@@ -388,4 +389,5 @@ bool BetaRowsetReader::_should_push_down_value_predicates() 
const {
              _read_context->sequence_id_idx == -1) ||
             _read_context->enable_unique_key_merge_on_write);
 }
+#include "common/compile_check_end.h"
 } // namespace doris
diff --git a/be/src/olap/rowset/segment_v2/segment_writer.cpp 
b/be/src/olap/rowset/segment_v2/segment_writer.cpp
index 6f92b54f43b..7abfcd65f08 100644
--- a/be/src/olap/rowset/segment_v2/segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_writer.cpp
@@ -616,7 +616,7 @@ Status 
SegmentWriter::append_block_with_partial_content(const vectorized::Block*
     // write including columns
     std::vector<vectorized::IOlapColumnDataAccessor*> key_columns;
     vectorized::IOlapColumnDataAccessor* seq_column = nullptr;
-    size_t segment_start_pos;
+    size_t segment_start_pos = 0;
     for (auto cid : including_cids) {
         // here we get segment column row num before append data.
         segment_start_pos = _column_writers[cid]->get_next_rowid();
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 a1105bb29f6..bfc35f2f439 100644
--- a/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
+++ b/be/src/olap/rowset/segment_v2/vertical_segment_writer.cpp
@@ -376,9 +376,9 @@ Status VerticalSegmentWriter::_probe_key_for_mow(
     RowLocation loc;
     // save rowset shared ptr so this rowset wouldn't delete
     RowsetSharedPtr rowset;
-    auto st = _tablet->lookup_row_key(
-            key, _tablet_schema.get(), have_input_seq_column, 
specified_rowsets, &loc,
-            cast_set<uint32_t>(_mow_context->max_version), segment_caches, 
&rowset);
+    auto st = _tablet->lookup_row_key(key, _tablet_schema.get(), 
have_input_seq_column,
+                                      specified_rowsets, &loc, 
_mow_context->max_version,
+                                      segment_caches, &rowset);
     if (st.is<KEY_NOT_FOUND>()) {
         if (!have_delete_sign) {
             RETURN_IF_ERROR(not_found_cb());
@@ -502,10 +502,10 @@ Status 
VerticalSegmentWriter::_append_block_with_partial_content(RowsInBlock& da
     // write including columns
     std::vector<vectorized::IOlapColumnDataAccessor*> key_columns;
     vectorized::IOlapColumnDataAccessor* seq_column = nullptr;
-    size_t segment_start_pos;
+    uint32_t segment_start_pos = 0;
     for (auto cid : including_cids) {
         // here we get segment column row num before append data.
-        segment_start_pos = _column_writers[cid]->get_next_rowid();
+        segment_start_pos = 
cast_set<uint32_t>(_column_writers[cid]->get_next_rowid());
         // olap data convertor alway start from id = 0
         auto [status, column] = _olap_data_convertor->convert_column_data(cid);
         if (!status.ok()) {
@@ -646,7 +646,7 @@ Status 
VerticalSegmentWriter::_append_block_with_flexible_partial_content(
     // create full block and fill with sort key columns
     full_block = _tablet_schema->create_block();
 
-    auto segment_start_pos = _column_writers.front()->get_next_rowid();
+    uint32_t segment_start_pos = 
cast_set<uint32_t>(_column_writers.front()->get_next_rowid());
 
     DCHECK(_tablet_schema->has_skip_bitmap_col());
     auto skip_bitmap_col_idx = _tablet_schema->skip_bitmap_col_idx();
@@ -725,8 +725,8 @@ Status 
VerticalSegmentWriter::_append_block_with_flexible_partial_content(
     // 6. read according plan to fill full_block
     RETURN_IF_ERROR(read_plan.fill_non_primary_key_columns(
             _opts.rowset_ctx, _rsid_to_rowset, *_tablet_schema, full_block,
-            use_default_or_null_flag, has_default_or_nullable, 
segment_start_pos, data.row_pos,
-            data.block, skip_bitmaps));
+            use_default_or_null_flag, has_default_or_nullable, 
segment_start_pos,
+            cast_set<uint32_t>(data.row_pos), data.block, skip_bitmaps));
 
     // TODO(bobhan1): should we replace the skip bitmap column with empty 
bitmaps to reduce storage occupation?
     // this column is not needed in read path for merge-on-write table


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


Reply via email to