From 9bd671f6b97952e998f42e5a8dbbdb71db687fa3 Mon Sep 17 00:00:00 2001 From: Mryange <59914473+Mryange@users.noreply.github.com> Date: Sun, 29 Sep 2024 11:55:16 +0800 Subject: [PATCH] [chore](check) open shorten-64-to-32 error (#41197) ``` doris/be/src/pipeline/pipeline_fragment_context.cpp:1160:76: error: implicit conversion loses integer precision: 'size_type' (aka 'unsigned long') to 'int' [-Werror,-Wshorten-64-to-32] 1159 | _sink.reset(new MultiCastDataStreamSinkOperatorX( | ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ 1160 | sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool, | ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~^~~~~~ ``` --- be/src/common/cast_set.h | 73 +++++++++++++++++++ be/src/common/compile_check_begin.h | 24 ++++++ be/src/common/compile_check_end.h | 23 ++++++ .../exec/aggregation_sink_operator.cpp | 18 ++--- .../exec/aggregation_source_operator.cpp | 18 ++--- .../pipeline/exec/analytic_sink_operator.cpp | 3 +- be/src/pipeline/exec/analytic_sink_operator.h | 2 +- .../exec/analytic_source_operator.cpp | 7 +- .../pipeline/exec/analytic_source_operator.h | 2 +- be/src/pipeline/exec/scan_operator.cpp | 3 +- be/src/pipeline/pipeline_fragment_context.cpp | 22 +++--- be/src/vec/core/block.cpp | 4 +- be/src/vec/core/block.h | 6 +- be/src/vec/exprs/vectorized_agg_fn.cpp | 2 +- be/src/vec/exprs/vectorized_agg_fn.h | 2 +- be/src/vec/exprs/vexpr_context.cpp | 4 +- be/src/vec/exprs/vexpr_context.h | 4 +- 17 files changed, 172 insertions(+), 45 deletions(-) create mode 100644 be/src/common/cast_set.h create mode 100644 be/src/common/compile_check_begin.h create mode 100644 be/src/common/compile_check_end.h diff --git a/be/src/common/cast_set.h b/be/src/common/cast_set.h new file mode 100644 index 00000000000000..08a9c8fbb7ebaa --- /dev/null +++ b/be/src/common/cast_set.h @@ -0,0 +1,73 @@ + +// 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 +#include + +#include "common/exception.h" +#include "common/status.h" + +namespace doris { + +template +void check_cast_value(U b) { + if constexpr (std::is_unsigned_v) { + if (b > std::numeric_limits::max()) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, + "value {} cast to type {} out of range [{},{}]", b, + typeid(T).name(), std::numeric_limits::min(), + std::numeric_limits::max()); + } + } else if constexpr (std::is_unsigned_v) { + if (b < 0 || b > std::numeric_limits::max()) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, + "value {} cast to type {} out of range [{},{}]", b, + typeid(T).name(), std::numeric_limits::min(), + std::numeric_limits::max()); + } + } else { + if (b < std::numeric_limits::min() || b > std::numeric_limits::max()) { + throw doris::Exception(ErrorCode::INVALID_ARGUMENT, + "value {} cast to type {} out of range [{},{}]", b, + typeid(T).name(), std::numeric_limits::min(), + std::numeric_limits::max()); + } + } +} + +template + requires std::is_integral_v && std::is_integral_v +void cast_set(T& a, U b) { + if constexpr (need_check_value) { + check_cast_value(b); + } + a = static_cast(b); +} + +template + requires std::is_integral_v && std::is_integral_v +T cast_set(U b) { + if constexpr (need_check_value) { + check_cast_value(b); + } + return static_cast(b); +} + +} // namespace doris diff --git a/be/src/common/compile_check_begin.h b/be/src/common/compile_check_begin.h new file mode 100644 index 00000000000000..8f5358fb89c6ed --- /dev/null +++ b/be/src/common/compile_check_begin.h @@ -0,0 +1,24 @@ +// 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 + +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic error "-Wshorten-64-to-32" +#endif +//#include "common/compile_check_begin.h" \ No newline at end of file diff --git a/be/src/common/compile_check_end.h b/be/src/common/compile_check_end.h new file mode 100644 index 00000000000000..491f41b6c27c1c --- /dev/null +++ b/be/src/common/compile_check_end.h @@ -0,0 +1,23 @@ +// 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 + +#ifdef __clang__ +#pragma clang diagnostic pop +#endif +// #include "common/compile_check_end.h" \ No newline at end of file diff --git a/be/src/pipeline/exec/aggregation_sink_operator.cpp b/be/src/pipeline/exec/aggregation_sink_operator.cpp index 260a599a947a0d..0bf3f8c4e0931d 100644 --- a/be/src/pipeline/exec/aggregation_sink_operator.cpp +++ b/be/src/pipeline/exec/aggregation_sink_operator.cpp @@ -27,7 +27,7 @@ #include "vec/exprs/vectorized_agg_fn.h" namespace doris::pipeline { - +#include "common/compile_check_begin.h" /// The minimum reduction factor (input rows divided by output rows) to grow hash tables /// in a streaming preaggregation, given that the hash tables are currently the given /// size or above. The sizes roughly correspond to hash table sizes where the bucket @@ -266,7 +266,7 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b vectorized::ColumnRawPtrs key_columns(key_size); std::vector key_locs(key_size); - for (size_t i = 0; i < key_size; ++i) { + for (int i = 0; i < key_size; ++i) { if constexpr (for_spill) { key_columns[i] = block->get_by_position(i).column.get(); key_locs[i] = i; @@ -279,7 +279,7 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b } } - int rows = block->rows(); + size_t rows = block->rows(); if (_places.size() < rows) { _places.resize(rows); } @@ -336,7 +336,7 @@ Status AggSinkLocalState::_merge_with_serialized_key_helper(vectorized::Block* b if (need_do_agg) { for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { if (Base::_shared_state->aggregate_evaluators[i]->is_merge() || for_spill) { - int col_id = 0; + size_t col_id = 0; if constexpr (for_spill) { col_id = Base::_shared_state->probe_expr_ctxs.size() + i; } else { @@ -459,7 +459,7 @@ Status AggSinkLocalState::_execute_with_serialized_key_helper(vectorized::Block* } } - int rows = block->rows(); + size_t rows = block->rows(); if (_places.size() < rows) { _places.resize(rows); } @@ -756,7 +756,7 @@ Status AggSinkOperatorX::init(const TPlanNode& tnode, RuntimeState* state) { const auto& agg_sort_info = tnode.agg_node.agg_sort_info_by_group_key; DCHECK_EQ(agg_sort_info.nulls_first.size(), agg_sort_info.is_asc_order.size()); - const int order_by_key_size = agg_sort_info.is_asc_order.size(); + const size_t order_by_key_size = agg_sort_info.is_asc_order.size(); _order_directions.resize(order_by_key_size); _null_directions.resize(order_by_key_size); for (int i = 0; i < order_by_key_size; ++i) { @@ -777,8 +777,8 @@ Status AggSinkOperatorX::open(RuntimeState* state) { RETURN_IF_ERROR(vectorized::VExpr::prepare( _probe_expr_ctxs, state, DataSinkOperatorX::_child->row_desc())); - int j = _probe_expr_ctxs.size(); - for (int i = 0; i < j; ++i) { + size_t j = _probe_expr_ctxs.size(); + for (size_t i = 0; i < j; ++i) { auto nullable_output = _output_tuple_desc->slots()[i]->is_nullable(); auto nullable_input = _probe_expr_ctxs[i]->root()->is_nullable(); if (nullable_output != nullable_input) { @@ -786,7 +786,7 @@ Status AggSinkOperatorX::open(RuntimeState* state) { _make_nullable_keys.emplace_back(i); } } - for (int i = 0; i < _aggregate_evaluators.size(); ++i, ++j) { + for (size_t i = 0; i < _aggregate_evaluators.size(); ++i, ++j) { SlotDescriptor* intermediate_slot_desc = _intermediate_tuple_desc->slots()[j]; SlotDescriptor* output_slot_desc = _output_tuple_desc->slots()[j]; RETURN_IF_ERROR(_aggregate_evaluators[i]->prepare( diff --git a/be/src/pipeline/exec/aggregation_source_operator.cpp b/be/src/pipeline/exec/aggregation_source_operator.cpp index fe03eba4102955..3bdda31308ff86 100644 --- a/be/src/pipeline/exec/aggregation_source_operator.cpp +++ b/be/src/pipeline/exec/aggregation_source_operator.cpp @@ -25,6 +25,7 @@ #include "vec/exprs/vectorized_agg_fn.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" AggLocalState::AggLocalState(RuntimeState* state, OperatorXBase* parent) : Base(state, parent), @@ -106,8 +107,8 @@ Status AggLocalState::_serialize_with_serialized_key_result(RuntimeState* state, vectorized::Block* block, bool* eos) { SCOPED_TIMER(_serialize_result_timer); auto& shared_state = *_shared_state; - int key_size = _shared_state->probe_expr_ctxs.size(); - int agg_size = _shared_state->aggregate_evaluators.size(); + size_t key_size = _shared_state->probe_expr_ctxs.size(); + size_t agg_size = _shared_state->aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); vectorized::DataTypes value_data_types(agg_size); @@ -229,7 +230,7 @@ Status AggLocalState::_get_with_serialized_key_result(RuntimeState* state, vecto auto columns_with_schema = vectorized::VectorizedUtils::create_columns_with_type_and_name( _parent->cast()._row_descriptor); - int key_size = shared_state.probe_expr_ctxs.size(); + size_t key_size = shared_state.probe_expr_ctxs.size(); vectorized::MutableColumns key_columns; for (int i = 0; i < key_size; ++i) { @@ -240,7 +241,7 @@ Status AggLocalState::_get_with_serialized_key_result(RuntimeState* state, vecto } } vectorized::MutableColumns value_columns; - for (int i = key_size; i < columns_with_schema.size(); ++i) { + for (size_t i = key_size; i < columns_with_schema.size(); ++i) { if (!mem_reuse) { value_columns.emplace_back(columns_with_schema[i].type->create_column()); } else { @@ -346,7 +347,7 @@ Status AggLocalState::_serialize_without_key(RuntimeState* state, vectorized::Bl block->clear(); DCHECK(shared_state.agg_data->without_key != nullptr); - int agg_size = shared_state.aggregate_evaluators.size(); + size_t agg_size = shared_state.aggregate_evaluators.size(); vectorized::MutableColumns value_columns(agg_size); std::vector data_types(agg_size); @@ -385,7 +386,7 @@ Status AggLocalState::_get_without_key_result(RuntimeState* state, vectorized::B auto& p = _parent->cast(); *block = vectorized::VectorizedUtils::create_empty_columnswithtypename(p._row_descriptor); - int agg_size = shared_state.aggregate_evaluators.size(); + size_t agg_size = shared_state.aggregate_evaluators.size(); vectorized::MutableColumns columns(agg_size); std::vector data_types(agg_size); @@ -495,7 +496,7 @@ Status AggLocalState::merge_with_serialized_key_helper(vectorized::Block* block) key_columns[i] = block->get_by_position(i).column.get(); } - int rows = block->rows(); + size_t rows = block->rows(); if (_places.size() < rows) { _places.resize(rows); } @@ -539,8 +540,7 @@ Status AggLocalState::merge_with_serialized_key_helper(vectorized::Block* block) _emplace_into_hash_table(_places.data(), key_columns, rows); for (int i = 0; i < Base::_shared_state->aggregate_evaluators.size(); ++i) { - int col_id = 0; - col_id = Base::_shared_state->probe_expr_ctxs.size() + i; + auto col_id = Base::_shared_state->probe_expr_ctxs.size() + i; auto column = block->get_by_position(col_id).column; if (column->is_nullable()) { column = ((vectorized::ColumnNullable*)column.get())->get_nested_column_ptr(); diff --git a/be/src/pipeline/exec/analytic_sink_operator.cpp b/be/src/pipeline/exec/analytic_sink_operator.cpp index 85d7773bdbd025..e9276e4fa048c8 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.cpp +++ b/be/src/pipeline/exec/analytic_sink_operator.cpp @@ -24,6 +24,7 @@ #include "vec/exprs/vectorized_agg_fn.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" Status AnalyticSinkLocalState::init(RuntimeState* state, LocalSinkStateInfo& info) { RETURN_IF_ERROR(PipelineXSinkLocalState::init(state, info)); @@ -91,7 +92,7 @@ bool AnalyticSinkLocalState::_whether_need_next_partition(BlockRowPos& found_par } //_partition_by_columns,_order_by_columns save in blocks, so if need to calculate the boundary, may find in which blocks firstly -BlockRowPos AnalyticSinkLocalState::_compare_row_to_find_end(int idx, BlockRowPos start, +BlockRowPos AnalyticSinkLocalState::_compare_row_to_find_end(int64_t idx, BlockRowPos start, BlockRowPos end, bool need_check_first) { auto& shared_state = *_shared_state; diff --git a/be/src/pipeline/exec/analytic_sink_operator.h b/be/src/pipeline/exec/analytic_sink_operator.h index b8615717198dd3..cf2892eb7e6ceb 100644 --- a/be/src/pipeline/exec/analytic_sink_operator.h +++ b/be/src/pipeline/exec/analytic_sink_operator.h @@ -53,7 +53,7 @@ class AnalyticSinkLocalState : public PipelineXSinkLocalState(state, parent), @@ -38,7 +39,7 @@ AnalyticLocalState::AnalyticLocalState(RuntimeState* state, OperatorXBase* paren _agg_arena_pool(std::make_unique()) {} //_partition_by_columns,_order_by_columns save in blocks, so if need to calculate the boundary, may find in which blocks firstly -BlockRowPos AnalyticLocalState::_compare_row_to_find_end(int idx, BlockRowPos start, +BlockRowPos AnalyticLocalState::_compare_row_to_find_end(int64_t idx, BlockRowPos start, BlockRowPos end, bool need_check_first) { auto& shared_state = *_shared_state; int64_t start_init_row_num = start.row_num; @@ -320,8 +321,8 @@ void AnalyticLocalState::_insert_result_info(int64_t current_block_rows) { const auto& offsets_of_aggregate_states = _parent->cast()._offsets_of_aggregate_states; - for (int i = 0; i < _agg_functions_size; ++i) { - for (int j = get_result_start; j < _window_end_position; ++j) { + for (size_t i = 0; i < _agg_functions_size; ++i) { + for (size_t j = get_result_start; j < _window_end_position; ++j) { if (!_agg_functions[i]->function()->get_return_type()->is_nullable() && _result_window_columns[i]->is_nullable()) { if (_current_window_empty) { diff --git a/be/src/pipeline/exec/analytic_source_operator.h b/be/src/pipeline/exec/analytic_source_operator.h index 38323f1b86bce2..d8dafa875dee6b 100644 --- a/be/src/pipeline/exec/analytic_source_operator.h +++ b/be/src/pipeline/exec/analytic_source_operator.h @@ -67,7 +67,7 @@ class AnalyticLocalState final : public PipelineXLocalState return need_more_input; } BlockRowPos _get_partition_by_end(); - BlockRowPos _compare_row_to_find_end(int idx, BlockRowPos start, BlockRowPos end, + BlockRowPos _compare_row_to_find_end(int64_t idx, BlockRowPos start, BlockRowPos end, bool need_check_first = false); bool _whether_need_next_partition(BlockRowPos& found_partition_end); diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index e46274398b5b9e..b81a64c7dfe5fb 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -45,6 +45,7 @@ namespace doris::pipeline { +#include "common/compile_check_begin.h" const static int32_t ADAPTIVE_PIPELINE_TASK_SERIAL_READ_ON_LIMIT_DEFAULT = 10000; #define RETURN_IF_PUSH_DOWN(stmt, status) \ @@ -682,7 +683,7 @@ Status ScanLocalState::_should_push_down_binary_predicate( DCHECK(constant_val->data == nullptr) << "constant_val should not have a value"; const auto& children = fn_call->children(); DCHECK(children.size() == 2); - for (size_t i = 0; i < children.size(); i++) { + for (int i = 0; i < 2; i++) { if (vectorized::VExpr::expr_without_cast(children[i])->node_type() != TExprNodeType::SLOT_REF) { // not a slot ref(column) diff --git a/be/src/pipeline/pipeline_fragment_context.cpp b/be/src/pipeline/pipeline_fragment_context.cpp index d17ddb6f3340c7..e99d8a17262e2e 100644 --- a/be/src/pipeline/pipeline_fragment_context.cpp +++ b/be/src/pipeline/pipeline_fragment_context.cpp @@ -33,6 +33,7 @@ #include #include "cloud/config.h" +#include "common/cast_set.h" #include "common/config.h" #include "common/logging.h" #include "common/status.h" @@ -112,6 +113,8 @@ #include "vec/runtime/vdata_stream_mgr.h" namespace doris::pipeline { +#include "common/compile_check_begin.h" +bvar::Adder g_pipeline_tasks_count("doris_pipeline_tasks_count"); PipelineFragmentContext::PipelineFragmentContext( const TUniqueId& query_id, const int fragment_id, std::shared_ptr query_ctx, @@ -243,7 +246,7 @@ Status PipelineFragmentContext::prepare(const doris::TPipelineFragmentParams& re _prepare_all_pipelines_timer = ADD_TIMER(_fragment_level_profile, "PrepareAllPipelinesTime"); { SCOPED_TIMER(_init_context_timer); - _num_instances = request.local_params.size(); + cast_set(_num_instances, request.local_params.size()); _total_instances = request.__isset.total_instances ? request.total_instances : _num_instances; @@ -365,7 +368,7 @@ Status PipelineFragmentContext::_build_pipeline_tasks(const doris::TPipelineFrag } auto pipeline_id_to_profile = _runtime_state->build_pipeline_profile(_pipelines.size()); - auto pre_and_submit = [&](int i, PipelineFragmentContext* ctx) { + auto pre_and_submit = [&](int64_t i, PipelineFragmentContext* ctx) { const auto& local_params = request.local_params[i]; auto fragment_instance_id = local_params.fragment_instance_id; _fragment_instance_ids[i] = fragment_instance_id; @@ -941,9 +944,10 @@ Status PipelineFragmentContext::_add_local_exchange( if (cur_pipe->num_tasks() > 1 && new_pip->num_tasks() == 1 && Pipeline::is_hash_exchange(data_distribution.distribution_type)) { RETURN_IF_ERROR(_add_local_exchange_impl( - new_pip->operators().size(), pool, new_pip, add_pipeline(new_pip, pip_idx + 2), - DataDistribution(ExchangeType::PASSTHROUGH), do_local_exchange, num_buckets, - bucket_seq_to_instance_idx, shuffle_idx_to_instance_idx, ignore_data_distribution)); + cast_set(new_pip->operators().size()), pool, new_pip, + add_pipeline(new_pip, pip_idx + 2), DataDistribution(ExchangeType::PASSTHROUGH), + do_local_exchange, num_buckets, bucket_seq_to_instance_idx, + shuffle_idx_to_instance_idx, ignore_data_distribution)); } return Status::OK(); } @@ -951,7 +955,7 @@ Status PipelineFragmentContext::_add_local_exchange( Status PipelineFragmentContext::_plan_local_exchange( int num_buckets, const std::map& bucket_seq_to_instance_idx, const std::map& shuffle_idx_to_instance_idx) { - for (int pip_idx = _pipelines.size() - 1; pip_idx >= 0; pip_idx--) { + for (int pip_idx = cast_set(_pipelines.size()) - 1; pip_idx >= 0; pip_idx--) { _pipelines[pip_idx]->init_data_distribution(); // Set property if child pipeline is not join operator's child. if (!_pipelines[pip_idx]->children().empty()) { @@ -1135,8 +1139,8 @@ Status PipelineFragmentContext::_create_data_sink(ObjectPool* pool, const TDataS } _sink.reset(new MultiCastDataStreamSinkOperatorX( - sink_id, sources, thrift_sink.multi_cast_stream_sink.sinks.size(), pool, - thrift_sink.multi_cast_stream_sink, row_desc)); + sink_id, sources, cast_set(thrift_sink.multi_cast_stream_sink.sinks.size()), + pool, thrift_sink.multi_cast_stream_sink, row_desc)); for (int i = 0; i < sender_size; ++i) { auto new_pipeline = add_pipeline(); RowDescriptor* _row_desc = nullptr; @@ -1870,5 +1874,5 @@ PipelineFragmentContext::collect_realtime_load_channel_profile() const { _runtime_state->load_channel_profile()->to_thrift(load_channel_profile.get()); return load_channel_profile; } - +#include "common/compile_check_end.h" } // namespace doris::pipeline diff --git a/be/src/vec/core/block.cpp b/be/src/vec/core/block.cpp index 971e48d52b6d54..1e245a8d31cc79 100644 --- a/be/src/vec/core/block.cpp +++ b/be/src/vec/core/block.cpp @@ -854,7 +854,7 @@ Status Block::append_to_block_by_selector(MutableBlock* dst, } Status Block::filter_block(Block* block, const std::vector& columns_to_filter, - int filter_column_id, int column_to_keep) { + size_t filter_column_id, size_t column_to_keep) { const auto& filter_column = block->get_by_position(filter_column_id).column; if (const auto* nullable_column = check_and_get_column(*filter_column)) { const auto& nested_column = nullable_column->get_nested_column_ptr(); @@ -892,7 +892,7 @@ Status Block::filter_block(Block* block, const std::vector& columns_to return Status::OK(); } -Status Block::filter_block(Block* block, int filter_column_id, int column_to_keep) { +Status Block::filter_block(Block* block, size_t filter_column_id, size_t column_to_keep) { std::vector columns_to_filter; columns_to_filter.resize(column_to_keep); for (uint32_t i = 0; i < column_to_keep; ++i) { diff --git a/be/src/vec/core/block.h b/be/src/vec/core/block.h index 108cb5e1c9fdf5..9abc75141290f5 100644 --- a/be/src/vec/core/block.h +++ b/be/src/vec/core/block.h @@ -294,11 +294,11 @@ class Block { static void filter_block_internal(Block* block, const IColumn::Filter& filter); static Status filter_block(Block* block, const std::vector& columns_to_filter, - int filter_column_id, int column_to_keep); + size_t filter_column_id, size_t column_to_keep); - static Status filter_block(Block* block, int filter_column_id, int column_to_keep); + static Status filter_block(Block* block, size_t filter_column_id, size_t column_to_keep); - static void erase_useless_column(Block* block, int column_to_keep) { + static void erase_useless_column(Block* block, size_t column_to_keep) { block->erase_tail(column_to_keep); } diff --git a/be/src/vec/exprs/vectorized_agg_fn.cpp b/be/src/vec/exprs/vectorized_agg_fn.cpp index c96d84db16c89c..4f40ff76ce59a0 100644 --- a/be/src/vec/exprs/vectorized_agg_fn.cpp +++ b/be/src/vec/exprs/vectorized_agg_fn.cpp @@ -351,7 +351,7 @@ AggFnEvaluator::AggFnEvaluator(AggFnEvaluator& evaluator, RuntimeState* state) } } -Status AggFnEvaluator::check_agg_fn_output(int key_size, +Status AggFnEvaluator::check_agg_fn_output(int64_t key_size, const std::vector& agg_fn, const RowDescriptor& output_row_desc) { auto name_and_types = VectorizedUtils::create_name_and_data_types(output_row_desc); diff --git a/be/src/vec/exprs/vectorized_agg_fn.h b/be/src/vec/exprs/vectorized_agg_fn.h index 7dcd1b3e02bb47..8e4f864c474058 100644 --- a/be/src/vec/exprs/vectorized_agg_fn.h +++ b/be/src/vec/exprs/vectorized_agg_fn.h @@ -97,7 +97,7 @@ class AggFnEvaluator { bool is_merge() const { return _is_merge; } const VExprContextSPtrs& input_exprs_ctxs() const { return _input_exprs_ctxs; } - static Status check_agg_fn_output(int key_size, + static Status check_agg_fn_output(int64_t key_size, const std::vector& agg_fn, const RowDescriptor& output_row_desc); diff --git a/be/src/vec/exprs/vexpr_context.cpp b/be/src/vec/exprs/vexpr_context.cpp index bcfd7cda102e2c..0afb1de51dee51 100644 --- a/be/src/vec/exprs/vexpr_context.cpp +++ b/be/src/vec/exprs/vexpr_context.cpp @@ -131,7 +131,7 @@ bool VExprContext::all_expr_inverted_index_evaluated() { return _inverted_index_context->has_inverted_index_result_for_expr(_root.get()); } -Status VExprContext::filter_block(VExprContext* vexpr_ctx, Block* block, int column_to_keep) { +Status VExprContext::filter_block(VExprContext* vexpr_ctx, Block* block, size_t column_to_keep) { if (vexpr_ctx == nullptr || block->rows() == 0) { return Status::OK(); } @@ -141,7 +141,7 @@ Status VExprContext::filter_block(VExprContext* vexpr_ctx, Block* block, int col } Status VExprContext::filter_block(const VExprContextSPtrs& expr_contexts, Block* block, - int column_to_keep) { + size_t column_to_keep) { if (expr_contexts.empty() || block->rows() == 0) { return Status::OK(); } diff --git a/be/src/vec/exprs/vexpr_context.h b/be/src/vec/exprs/vexpr_context.h index d022cf6169e26a..d3b159fc400ecc 100644 --- a/be/src/vec/exprs/vexpr_context.h +++ b/be/src/vec/exprs/vexpr_context.h @@ -194,10 +194,10 @@ class VExprContext { bool all_expr_inverted_index_evaluated(); [[nodiscard]] static Status filter_block(VExprContext* vexpr_ctx, Block* block, - int column_to_keep); + size_t column_to_keep); [[nodiscard]] static Status filter_block(const VExprContextSPtrs& expr_contexts, Block* block, - int column_to_keep); + size_t column_to_keep); [[nodiscard]] static Status execute_conjuncts(const VExprContextSPtrs& ctxs, const std::vector* filters,