This is an automated email from the ASF dual-hosted git repository. panxiaolei 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 208fde06481 [refine](Column)Disallow implicit conversion of ColumnPtr to IColumn* (#45588) 208fde06481 is described below commit 208fde0648110f3a6d2837b43bbd8c19c5c51d33 Author: Mryange <yanxuech...@selectdb.com> AuthorDate: Mon Dec 23 11:33:56 2024 +0800 [refine](Column)Disallow implicit conversion of ColumnPtr to IColumn* (#45588) ### What problem does this PR solve? Previously, we allowed ColumnPtr to be directly converted to Column*: ```C++ ColumnPtr column; const IColumn* ptr = column; ``` This can easily cause confusion. For example, in the following code: ```C++ ColumnPtr column; const auto& const_column = check_and_get_column<ColumnConst>(column); ``` The matched function is: ```C++ template <> const doris::vectorized::ColumnConst* check_and_get_column<doris::vectorized::ColumnConst>( const IColumn* column) ``` However, the actual type of const_column is: ```C++ const doris::vectorized::ColumnConst* const& ``` ### 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) - [x] No need to test or manual test. Explain why: - [x] 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: - [x] No. - [ ] Yes. <!-- Explain the behavior change --> - Does this need documentation? - [x] 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/table_connector.cpp | 7 ++-- be/src/olap/push_handler.cpp | 2 +- be/src/olap/rowset/segment_v2/column_reader.cpp | 13 +++---- .../rowset/segment_v2/hierarchical_data_reader.h | 4 +-- be/src/olap/rowset/segment_v2/segment_iterator.cpp | 5 ++- be/src/olap/schema_change.cpp | 14 ++++---- be/src/pipeline/exec/hashjoin_build_sink.cpp | 2 +- be/src/pipeline/exec/hashjoin_probe_operator.cpp | 4 +-- be/src/pipeline/exec/join_probe_operator.cpp | 2 +- be/src/pipeline/exec/olap_scan_operator.cpp | 5 ++- be/src/pipeline/exec/scan_operator.cpp | 8 ++--- .../aggregate_function_window.h | 2 +- be/src/vec/columns/column_object.cpp | 6 ++-- be/src/vec/common/cow.h | 6 ++-- be/src/vec/exec/format/column_type_convert.cpp | 2 +- be/src/vec/exec/format/csv/csv_reader.cpp | 4 +-- be/src/vec/exec/format/json/new_json_reader.cpp | 23 ++++++------ .../exec/format/parquet/parquet_column_convert.cpp | 2 +- be/src/vec/exec/format/wal/wal_reader.cpp | 2 +- be/src/vec/exec/jni_connector.cpp | 2 +- be/src/vec/exec/scan/vfile_scanner.cpp | 2 +- be/src/vec/exec/scan/vmeta_scanner.cpp | 2 +- be/src/vec/exprs/vcompound_pred.h | 6 ++-- .../vec/functions/array/function_array_cum_sum.cpp | 2 +- .../vec/functions/array/function_array_distinct.h | 4 +-- .../functions/array/function_array_enumerate.cpp | 5 +-- .../array/function_array_enumerate_uniq.cpp | 7 ++-- be/src/vec/functions/array/function_array_join.h | 7 ++-- be/src/vec/functions/array/function_array_map.h | 2 +- be/src/vec/functions/array/function_array_pop.cpp | 2 +- .../vec/functions/array/function_array_range.cpp | 2 +- be/src/vec/functions/array/function_array_remove.h | 8 ++--- .../vec/functions/array/function_array_reverse.h | 2 +- be/src/vec/functions/array/function_array_set.h | 4 +-- be/src/vec/functions/array/function_array_slice.h | 2 +- .../vec/functions/array/function_array_sortby.cpp | 6 ++-- .../vec/functions/array/function_arrays_overlap.h | 8 ++--- be/src/vec/functions/comparison_equal_for_null.cpp | 10 +++--- be/src/vec/functions/function_agg_state.h | 2 +- be/src/vec/functions/function_binary_arithmetic.h | 28 ++++++++------- be/src/vec/functions/function_bitmap.cpp | 2 +- be/src/vec/functions/function_bitmap_variadic.cpp | 2 +- be/src/vec/functions/function_case.h | 2 +- be/src/vec/functions/function_cast.h | 2 +- be/src/vec/functions/function_collection_in.h | 5 +-- .../function_date_or_datetime_computation.h | 4 +-- be/src/vec/functions/function_ip.h | 6 ++-- be/src/vec/functions/function_jsonb.cpp | 26 ++++++++------ be/src/vec/functions/function_nullables.cpp | 5 +-- be/src/vec/functions/function_quantile_state.cpp | 4 +-- be/src/vec/functions/function_string.h | 4 +-- be/src/vec/functions/function_tokenize.cpp | 2 +- be/src/vec/functions/functions_geo.cpp | 16 ++++----- be/src/vec/functions/functions_logical.cpp | 13 +++---- be/src/vec/functions/in.h | 2 +- be/src/vec/functions/least_greast.cpp | 2 +- be/src/vec/functions/round.h | 1 + be/src/vec/sink/vtablet_block_convertor.cpp | 5 +-- .../sink/writer/iceberg/partition_transformers.h | 42 +++++++++++----------- .../sink/writer/iceberg/viceberg_table_writer.cpp | 2 +- be/src/vec/utils/util.hpp | 2 +- be/test/vec/columns/common_column_test.h | 6 ++-- be/test/vec/data_types/from_string_test.cpp | 12 +++---- .../data_types/serde/data_type_serde_text_test.cpp | 12 +++---- be/test/vec/function/function_test_util.h | 2 +- be/test/vec/olap/char_type_padding_test.cpp | 6 ++-- 66 files changed, 218 insertions(+), 195 deletions(-) diff --git a/be/src/exec/table_connector.cpp b/be/src/exec/table_connector.cpp index fa5181f5fec..549fa6aae90 100644 --- a/be/src/exec/table_connector.cpp +++ b/be/src/exec/table_connector.cpp @@ -118,16 +118,17 @@ Status TableConnector::convert_column_data(const vectorized::ColumnPtr& column_p fmt::format_to(_insert_stmt_buffer, "\"{}\"", str); } }; - const vectorized::IColumn* column = column_ptr; + const vectorized::IColumn* column = column_ptr.get(); if (type_ptr->is_nullable()) { - auto nullable_column = assert_cast<const vectorized::ColumnNullable*>(column_ptr.get()); + const auto* nullable_column = + assert_cast<const vectorized::ColumnNullable*>(column_ptr.get()); if (nullable_column->is_null_at(row)) { fmt::format_to(_insert_stmt_buffer, "{}", "NULL"); return Status::OK(); } column = nullable_column->get_nested_column_ptr().get(); } else { - column = column_ptr; + column = column_ptr.get(); } auto [item, size] = column->get_data_at(row); switch (type.type) { diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp index 56d167459f5..eecb322384b 100644 --- a/be/src/olap/push_handler.cpp +++ b/be/src/olap/push_handler.cpp @@ -518,7 +518,7 @@ Status PushBrokerReader::_convert_to_output_block(vectorized::Block* block) { column_ptr = _src_block.get_by_position(result_column_id).column; // column_ptr maybe a ColumnConst, convert it to a normal column column_ptr = column_ptr->convert_to_full_column_if_const(); - DCHECK(column_ptr != nullptr); + DCHECK(column_ptr); // because of src_slot_desc is always be nullable, so the column_ptr after do dest_expr // is likely to be nullable diff --git a/be/src/olap/rowset/segment_v2/column_reader.cpp b/be/src/olap/rowset/segment_v2/column_reader.cpp index 78c415530cd..1abb60e5850 100644 --- a/be/src/olap/rowset/segment_v2/column_reader.cpp +++ b/be/src/olap/rowset/segment_v2/column_reader.cpp @@ -1267,8 +1267,8 @@ Status FileColumnIterator::next_batch(size_t* n, vectorized::MutableColumnPtr& d DCHECK_EQ(this_run, num_rows); } else { *has_null = true; - auto* null_col = - vectorized::check_and_get_column<vectorized::ColumnNullable>(dst); + const auto* null_col = + vectorized::check_and_get_column<vectorized::ColumnNullable>(dst.get()); if (null_col != nullptr) { const_cast<vectorized::ColumnNullable*>(null_col)->insert_null_elements( this_run); @@ -1328,8 +1328,9 @@ Status FileColumnIterator::read_by_rowids(const rowid_t* rowids, const size_t co auto origin_index = _page.data_decoder->current_index(); if (this_read_count > 0) { if (is_null) { - auto* null_col = - vectorized::check_and_get_column<vectorized::ColumnNullable>(dst); + const auto* null_col = + vectorized::check_and_get_column<vectorized::ColumnNullable>( + dst.get()); if (UNLIKELY(null_col == nullptr)) { return Status::InternalError("unexpected column type in column reader"); } @@ -1710,9 +1711,9 @@ Status DefaultNestedColumnIterator::next_batch(size_t* n, vectorized::MutableCol static void fill_nested_with_defaults(vectorized::MutableColumnPtr& dst, vectorized::MutableColumnPtr& sibling_column, size_t nrows) { const auto* sibling_array = vectorized::check_and_get_column<vectorized::ColumnArray>( - remove_nullable(sibling_column->get_ptr())); + remove_nullable(sibling_column->get_ptr()).get()); const auto* dst_array = vectorized::check_and_get_column<vectorized::ColumnArray>( - remove_nullable(dst->get_ptr())); + remove_nullable(dst->get_ptr()).get()); if (!dst_array || !sibling_array) { throw doris::Exception(ErrorCode::INTERNAL_ERROR, "Expected array column, but met %s and %s", dst->get_name(), diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h index f85038713ca..bd5de748474 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.h @@ -165,8 +165,8 @@ private: // will type the type of ColumnObject::NESTED_TYPE, whih is Nullable<ColumnArray<NULLABLE(ColumnObject)>>. for (auto& entry : nested_subcolumns) { MutableColumnPtr nested_object = ColumnObject::create(true, false); - const auto* base_array = - check_and_get_column<ColumnArray>(remove_nullable(entry.second[0].column)); + const auto* base_array = check_and_get_column<ColumnArray>( + remove_nullable(entry.second[0].column).get()); MutableColumnPtr offset = base_array->get_offsets_ptr()->assume_mutable(); auto* nested_object_ptr = assert_cast<ColumnObject*>(nested_object.get()); // flatten nested arrays diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp index 5f50ffeea2d..366c6d3ce21 100644 --- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp +++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp @@ -1955,8 +1955,7 @@ Status SegmentIterator::next_batch(vectorized::Block* block) { Status SegmentIterator::_convert_to_expected_type(const std::vector<ColumnId>& col_ids) { for (ColumnId i : col_ids) { - if (_current_return_columns[i] == nullptr || _converted_column_ids[i] || - _is_pred_column[i]) { + if (!_current_return_columns[i] || _converted_column_ids[i] || _is_pred_column[i]) { continue; } if (!_segment->same_with_storage_type( @@ -1999,7 +1998,7 @@ Status SegmentIterator::copy_column_data_by_selector(vectorized::IColumn* input_ return Status::RuntimeError("copy_column_data_by_selector nullable mismatch"); } - return input_col_ptr->filter_by_selector(sel_rowid_idx, select_size, output_col); + return input_col_ptr->filter_by_selector(sel_rowid_idx, select_size, output_col.get()); } void SegmentIterator::_clear_iterators() { diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp index 7f947612eed..658ff05b67f 100644 --- a/be/src/olap/schema_change.cpp +++ b/be/src/olap/schema_change.cpp @@ -337,7 +337,7 @@ Status BlockChanger::change_block(vectorized::Block* ref_block, int result_tmp_column_idx = -1; RETURN_IF_ERROR(ctx->execute(ref_block, &result_tmp_column_idx)); auto& result_tmp_column_def = ref_block->get_by_position(result_tmp_column_idx); - if (result_tmp_column_def.column == nullptr) { + if (!result_tmp_column_def.column) { return Status::Error<ErrorCode::INTERNAL_ERROR>( "result column={} is nullptr, input expr={}", result_tmp_column_def.name, apache::thrift::ThriftDebugString(*expr)); @@ -430,7 +430,7 @@ Status BlockChanger::_check_cast_valid(vectorized::ColumnPtr input_column, if (input_column->is_nullable() != output_column->is_nullable()) { if (input_column->is_nullable()) { const auto* ref_null_map = - vectorized::check_and_get_column<vectorized::ColumnNullable>(input_column) + vectorized::check_and_get_column<vectorized::ColumnNullable>(input_column.get()) ->get_null_map_column() .get_data() .data(); @@ -446,10 +446,12 @@ Status BlockChanger::_check_cast_valid(vectorized::ColumnPtr input_column, } } else { const auto& null_map_column = - vectorized::check_and_get_column<vectorized::ColumnNullable>(output_column) + vectorized::check_and_get_column<vectorized::ColumnNullable>( + output_column.get()) ->get_null_map_column(); const auto& nested_column = - vectorized::check_and_get_column<vectorized::ColumnNullable>(output_column) + vectorized::check_and_get_column<vectorized::ColumnNullable>( + output_column.get()) ->get_nested_column(); const auto* new_null_map = null_map_column.get_data().data(); @@ -481,12 +483,12 @@ Status BlockChanger::_check_cast_valid(vectorized::ColumnPtr input_column, if (input_column->is_nullable() && output_column->is_nullable()) { const auto* ref_null_map = - vectorized::check_and_get_column<vectorized::ColumnNullable>(input_column) + vectorized::check_and_get_column<vectorized::ColumnNullable>(input_column.get()) ->get_null_map_column() .get_data() .data(); const auto* new_null_map = - vectorized::check_and_get_column<vectorized::ColumnNullable>(output_column) + vectorized::check_and_get_column<vectorized::ColumnNullable>(output_column.get()) ->get_null_map_column() .get_data() .data(); diff --git a/be/src/pipeline/exec/hashjoin_build_sink.cpp b/be/src/pipeline/exec/hashjoin_build_sink.cpp index 47560875b51..b71feff3ed4 100644 --- a/be/src/pipeline/exec/hashjoin_build_sink.cpp +++ b/be/src/pipeline/exec/hashjoin_build_sink.cpp @@ -254,7 +254,7 @@ Status HashJoinBuildSinkLocalState::_extract_join_column( // update nulllmap and split nested out of ColumnNullable when serialize_null_into_key is false and column is nullable const auto& col_nested = nullable->get_nested_column(); const auto& col_nullmap = nullable->get_null_map_data(); - DCHECK(null_map != nullptr); + DCHECK(null_map); vectorized::VectorizedUtils::update_null_map(null_map->get_data(), col_nullmap); raw_ptrs[i] = &col_nested; } else { diff --git a/be/src/pipeline/exec/hashjoin_probe_operator.cpp b/be/src/pipeline/exec/hashjoin_probe_operator.cpp index 0db525f1bf5..37ccd6206f3 100644 --- a/be/src/pipeline/exec/hashjoin_probe_operator.cpp +++ b/be/src/pipeline/exec/hashjoin_probe_operator.cpp @@ -371,7 +371,7 @@ Status HashJoinProbeLocalState::_extract_join_column(vectorized::Block& block, _need_null_map_for_probe = _need_probe_null_map(block, res_col_ids); } if (_need_null_map_for_probe) { - if (_null_map_column == nullptr) { + if (!_null_map_column) { _null_map_column = vectorized::ColumnUInt8::create(); } _null_map_column->get_data().assign(block.rows(), (uint8_t)0); @@ -389,7 +389,7 @@ Status HashJoinProbeLocalState::_extract_join_column(vectorized::Block& block, // update nulllmap and split nested out of ColumnNullable when serialize_null_into_key is false and column is nullable const auto& col_nested = nullable->get_nested_column(); const auto& col_nullmap = nullable->get_null_map_data(); - DCHECK(_null_map_column != nullptr); + DCHECK(_null_map_column); vectorized::VectorizedUtils::update_null_map(_null_map_column->get_data(), col_nullmap); _probe_columns[i] = &col_nested; } else { diff --git a/be/src/pipeline/exec/join_probe_operator.cpp b/be/src/pipeline/exec/join_probe_operator.cpp index 11b5b29c8b5..9a50d76a48c 100644 --- a/be/src/pipeline/exec/join_probe_operator.cpp +++ b/be/src/pipeline/exec/join_probe_operator.cpp @@ -150,7 +150,7 @@ Status JoinProbeLocalState<SharedStateArg, Derived>::_build_output_block( /// TODO: maybe need a method to check if a column need to be converted to full /// column. if (is_column_const(*origin_column) || - check_column<vectorized::ColumnArray>(origin_column)) { + check_column<vectorized::ColumnArray>(origin_column.get())) { auto column_ptr = origin_column->convert_to_full_column_if_const(); insert_column_datas(mutable_columns[i], column_ptr, rows); } else { diff --git a/be/src/pipeline/exec/olap_scan_operator.cpp b/be/src/pipeline/exec/olap_scan_operator.cpp index fa91caffa8e..b1ab6274332 100644 --- a/be/src/pipeline/exec/olap_scan_operator.cpp +++ b/be/src/pipeline/exec/olap_scan_operator.cpp @@ -246,9 +246,8 @@ Status OlapScanLocalState::_should_push_down_function_filter(vectorized::Vectori DCHECK(children[1 - i]->type().is_string_type()); std::shared_ptr<ColumnPtrWrapper> const_col_wrapper; RETURN_IF_ERROR(children[1 - i]->get_const_col(expr_ctx, &const_col_wrapper)); - if (const vectorized::ColumnConst* const_column = - check_and_get_column<vectorized::ColumnConst>( - const_col_wrapper->column_ptr)) { + if (const auto* const_column = check_and_get_column<vectorized::ColumnConst>( + const_col_wrapper->column_ptr.get())) { *constant_str = const_column->get_data_at(0); } else { pdt = PushDownType::UNACCEPTABLE; diff --git a/be/src/pipeline/exec/scan_operator.cpp b/be/src/pipeline/exec/scan_operator.cpp index ae4396b22c7..a73e1a6db7c 100644 --- a/be/src/pipeline/exec/scan_operator.cpp +++ b/be/src/pipeline/exec/scan_operator.cpp @@ -520,8 +520,8 @@ Status ScanLocalState<Derived>::_eval_const_conjuncts(vectorized::VExpr* vexpr, if (vexpr->is_constant()) { std::shared_ptr<ColumnPtrWrapper> const_col_wrapper; RETURN_IF_ERROR(vexpr->get_const_col(expr_ctx, &const_col_wrapper)); - if (const auto* const_column = - check_and_get_column<vectorized::ColumnConst>(const_col_wrapper->column_ptr)) { + if (const auto* const_column = check_and_get_column<vectorized::ColumnConst>( + const_col_wrapper->column_ptr.get())) { constant_val = const_cast<char*>(const_column->get_data_at(0).data); if (constant_val == nullptr || !*reinterpret_cast<bool*>(constant_val)) { *pdt = PushDownType::ACCEPTABLE; @@ -530,7 +530,7 @@ Status ScanLocalState<Derived>::_eval_const_conjuncts(vectorized::VExpr* vexpr, } } else if (const auto* bool_column = check_and_get_column<vectorized::ColumnVector<vectorized::UInt8>>( - const_col_wrapper->column_ptr)) { + const_col_wrapper->column_ptr.get())) { // TODO: If `vexpr->is_constant()` is true, a const column is expected here. // But now we still don't cover all predicates for const expression. // For example, for query `SELECT col FROM tbl WHERE 'PROMOTION' LIKE 'AAA%'`, @@ -690,7 +690,7 @@ Status ScanLocalState<Derived>::_should_push_down_binary_predicate( std::shared_ptr<ColumnPtrWrapper> const_col_wrapper; RETURN_IF_ERROR(children[1 - i]->get_const_col(expr_ctx, &const_col_wrapper)); if (const auto* const_column = check_and_get_column<vectorized::ColumnConst>( - const_col_wrapper->column_ptr)) { + const_col_wrapper->column_ptr.get())) { *slot_ref_child = i; *constant_val = const_column->get_data_at(0); } else { diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h index 0cef4c82d3d..5d449318b7d 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_window.h +++ b/be/src/vec/aggregate_functions/aggregate_function_window.h @@ -402,7 +402,7 @@ public: if (nullable_column->is_null_at(0)) { _default_value.reset(); } else { - _default_value.set_value(nullable_column->get_nested_column_ptr(), 0); + _default_value.set_value(nullable_column->get_nested_column_ptr().get(), 0); } } else { _default_value.set_value(column, 0); diff --git a/be/src/vec/columns/column_object.cpp b/be/src/vec/columns/column_object.cpp index 3d6a3e44436..4300725caca 100644 --- a/be/src/vec/columns/column_object.cpp +++ b/be/src/vec/columns/column_object.cpp @@ -1484,7 +1484,7 @@ Status ColumnObject::serialize_one_row_to_json_format(size_t row, rapidjson::Str #endif for (const auto& subcolumn : subcolumns) { RETURN_IF_ERROR(find_and_set_leave_value( - subcolumn->data.get_finalized_column_ptr(), subcolumn->path, + subcolumn->data.get_finalized_column_ptr().get(), subcolumn->path, subcolumn->data.get_least_common_type_serde(), subcolumn->data.get_least_common_type(), subcolumn->data.least_common_type.get_base_type_id(), root, @@ -1558,7 +1558,7 @@ Status ColumnObject::merge_sparse_to_root_column() { continue; } bool succ = find_and_set_leave_value( - column, subcolumn->path, subcolumn->data.get_least_common_type_serde(), + column.get(), subcolumn->path, subcolumn->data.get_least_common_type_serde(), subcolumn->data.get_least_common_type(), subcolumn->data.least_common_type.get_base_type_id(), root, doc_structure->GetAllocator(), mem_pool, i); @@ -1705,7 +1705,7 @@ bool ColumnObject::empty() const { } ColumnPtr get_base_column_of_array(const ColumnPtr& column) { - if (const auto* column_array = check_and_get_column<ColumnArray>(column)) { + if (const auto* column_array = check_and_get_column<ColumnArray>(column.get())) { return column_array->get_data_ptr(); } return column; diff --git a/be/src/vec/common/cow.h b/be/src/vec/common/cow.h index 95df7694f22..4970f649d32 100644 --- a/be/src/vec/common/cow.h +++ b/be/src/vec/common/cow.h @@ -203,8 +203,6 @@ protected: operator bool() const { return t != nullptr; } - operator T*() const { return t; } - private: T* t = nullptr; }; @@ -346,8 +344,8 @@ protected: operator const immutable_ptr<T>&() const { return value; } operator immutable_ptr<T>&() { return value; } - operator bool() const { return value != nullptr; } - bool operator!() const { return value == nullptr; } + operator bool() const { return value.get() != nullptr; } + bool operator!() const { return value.get() == nullptr; } bool operator==(const chameleon_ptr& rhs) const { return value == rhs.value; } bool operator!=(const chameleon_ptr& rhs) const { return value != rhs.value; } diff --git a/be/src/vec/exec/format/column_type_convert.cpp b/be/src/vec/exec/format/column_type_convert.cpp index a2c226c91d6..0442158b690 100644 --- a/be/src/vec/exec/format/column_type_convert.cpp +++ b/be/src/vec/exec/format/column_type_convert.cpp @@ -99,7 +99,7 @@ ColumnPtr ColumnTypeConverter::get_column(const TypeDescriptor& src_type, Column return dst_column; } - if (_cached_src_column == nullptr) { + if (!_cached_src_column) { _cached_src_type = DataTypeFactory::instance().create_data_type(src_type, dst_type->is_nullable()); _cached_src_column = diff --git a/be/src/vec/exec/format/csv/csv_reader.cpp b/be/src/vec/exec/format/csv/csv_reader.cpp index b27bb050dc6..d4a2dcfc7f3 100644 --- a/be/src/vec/exec/format/csv/csv_reader.cpp +++ b/be/src/vec/exec/format/csv/csv_reader.cpp @@ -657,7 +657,7 @@ Status CsvReader::_fill_dest_columns(const Slice& line, Block* block, col_idx < _split_values.size() ? _split_values[col_idx] : _s_null_slice; Slice slice {value.data, value.size}; - IColumn* col_ptr = columns[i]; + IColumn* col_ptr = columns[i].get(); if (!_is_load) { col_ptr = const_cast<IColumn*>( block->get_by_position(_file_slot_idx_map[i]).column.get()); @@ -700,7 +700,7 @@ Status CsvReader::_fill_dest_columns(const Slice& line, Block* block, Status CsvReader::_fill_empty_line(Block* block, std::vector<MutableColumnPtr>& columns, size_t* rows) { for (int i = 0; i < _file_slot_descs.size(); ++i) { - IColumn* col_ptr = columns[i]; + IColumn* col_ptr = columns[i].get(); if (!_is_load) { col_ptr = const_cast<IColumn*>( block->get_by_position(_file_slot_idx_map[i]).column.get()); diff --git a/be/src/vec/exec/format/json/new_json_reader.cpp b/be/src/vec/exec/format/json/new_json_reader.cpp index d79e8652074..adb22d588f5 100644 --- a/be/src/vec/exec/format/json/new_json_reader.cpp +++ b/be/src/vec/exec/format/json/new_json_reader.cpp @@ -886,7 +886,7 @@ Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator if (column_ptr->is_nullable()) { nullable_column = reinterpret_cast<ColumnNullable*>(column_ptr); - data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_column_ptr = nullable_column->get_nested_column().get_ptr().get(); data_serde = serde->get_nested_serdes()[0]; if (value_is_null) { @@ -1010,7 +1010,8 @@ Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator const auto& sub_col_type = type_desc.children[sub_col_idx]; RETURN_IF_ERROR(_write_data_to_column( - sub_value, sub_col_type, struct_column_ptr->get_column(sub_col_idx).get_ptr(), + sub_value, sub_col_type, + struct_column_ptr->get_column(sub_col_idx).get_ptr().get(), column_name + "." + type_desc.field_names[sub_col_idx], sub_serdes[sub_col_idx], valid)); } @@ -1026,12 +1027,12 @@ Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator for (const auto& member_value : object_value) { RETURN_IF_ERROR(_write_data_to_column( &member_value.name, type_desc.children[0], - map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr().get(), column_name + ".key", sub_serdes[0], valid)); RETURN_IF_ERROR(_write_data_to_column( &member_value.value, type_desc.children[1], - map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr().get(), column_name + ".value", sub_serdes[1], valid)); } @@ -1048,7 +1049,7 @@ Status NewJsonReader::_write_data_to_column(rapidjson::Value::ConstValueIterator for (const auto& sub_value : array_value) { RETURN_IF_ERROR(_write_data_to_column(&sub_value, type_desc.children[0], - array_column_ptr->get_data().get_ptr(), + array_column_ptr->get_data().get_ptr().get(), column_name + ".element", sub_serdes[0], valid)); } auto& offsets = array_column_ptr->get_offsets(); @@ -1653,7 +1654,7 @@ Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& if (column_ptr->is_nullable()) { nullable_column = reinterpret_cast<ColumnNullable*>(column_ptr); - data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_column_ptr = nullable_column->get_nested_column().get_ptr().get(); data_serde = serde->get_nested_serdes()[0]; // kNullType will put 1 into the Null map, so there is no need to push 0 for kNullType. @@ -1727,7 +1728,7 @@ Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& const auto& sub_col_type = type_desc.children[sub_column_idx]; RETURN_IF_ERROR(_simdjson_write_data_to_column( - sub.value(), sub_col_type, sub_column_ptr, column_name + "." + sub_key, + sub.value(), sub_col_type, sub_column_ptr.get(), column_name + "." + sub_key, sub_serdes[sub_column_idx], valid)); } @@ -1768,7 +1769,7 @@ Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& auto nullable_column = static_cast<ColumnNullable*>(column_ptr); nullable_column->get_null_map_data().push_back(0); - data_column_ptr = nullable_column->get_nested_column().get_ptr(); + data_column_ptr = nullable_column->get_nested_column().get_ptr().get(); data_serde = serde->get_nested_serdes()[0]; } Slice slice(key_view.data(), key_view.length()); @@ -1779,13 +1780,13 @@ Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& }; RETURN_IF_ERROR(f(member_value.unescaped_key(), type_desc.children[0], - map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr(), + map_column_ptr->get_keys_ptr()->assume_mutable()->get_ptr().get(), sub_serdes[0], _serde_options, valid)); simdjson::ondemand::value field_value = member_value.value(); RETURN_IF_ERROR(_simdjson_write_data_to_column( field_value, type_desc.children[1], - map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr(), + map_column_ptr->get_values_ptr()->assume_mutable()->get_ptr().get(), column_name + ".value", sub_serdes[1], valid)); field_count++; } @@ -1807,7 +1808,7 @@ Status NewJsonReader::_simdjson_write_data_to_column(simdjson::ondemand::value& int field_count = 0; for (simdjson::ondemand::value sub_value : array_value) { RETURN_IF_ERROR(_simdjson_write_data_to_column( - sub_value, type_desc.children[0], array_column_ptr->get_data().get_ptr(), + sub_value, type_desc.children[0], array_column_ptr->get_data().get_ptr().get(), column_name + ".element", sub_serdes[0], valid)); field_count++; } diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp index 0a5ef2913dd..49636d809aa 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp @@ -79,7 +79,7 @@ ColumnPtr PhysicalToLogicalConverter::get_physical_column(tparquet::Type::type s return dst_logical_column; } - if (_cached_src_physical_column == nullptr) { + if (!_cached_src_physical_column) { switch (src_physical_type) { case tparquet::Type::type::BOOLEAN: _cached_src_physical_type = std::make_shared<DataTypeUInt8>(); diff --git a/be/src/vec/exec/format/wal/wal_reader.cpp b/be/src/vec/exec/format/wal/wal_reader.cpp index 22e6928216e..a9a209b95a4 100644 --- a/be/src/vec/exec/format/wal/wal_reader.cpp +++ b/be/src/vec/exec/format/wal/wal_reader.cpp @@ -92,7 +92,7 @@ Status WalReader::get_next_block(Block* block, size_t* read_rows, bool* eof) { pos, src_block.columns()); } vectorized::ColumnPtr column_ptr = src_block.get_by_position(pos).column; - if (column_ptr != nullptr && slot_desc->is_nullable()) { + if (!column_ptr && slot_desc->is_nullable()) { column_ptr = make_nullable(column_ptr); } dst_block.insert(index, vectorized::ColumnWithTypeAndName( diff --git a/be/src/vec/exec/jni_connector.cpp b/be/src/vec/exec/jni_connector.cpp index 11a58e81c98..4b5bb72e57b 100644 --- a/be/src/vec/exec/jni_connector.cpp +++ b/be/src/vec/exec/jni_connector.cpp @@ -241,7 +241,7 @@ Status JniConnector::fill_block(Block* block, const ColumnNumbers& arguments, lo TableMetaAddress table_meta(table_address); long num_rows = table_meta.next_meta_as_long(); for (size_t i : arguments) { - if (block->get_by_position(i).column == nullptr) { + if (block->get_by_position(i).column.get() == nullptr) { auto return_type = block->get_data_type(i); bool result_nullable = return_type->is_nullable(); ColumnUInt8::MutablePtr null_col = nullptr; diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 93a22d1a94b..15b681f5979 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -596,7 +596,7 @@ Status VFileScanner::_convert_to_output_block(Block* block) { column_ptr = _src_block_ptr->get_by_position(result_column_id).column; // column_ptr maybe a ColumnConst, convert it to a normal column column_ptr = column_ptr->convert_to_full_column_if_const(); - DCHECK(column_ptr != nullptr); + DCHECK(column_ptr); // because of src_slot_desc is always be nullable, so the column_ptr after do dest_expr // is likely to be nullable diff --git a/be/src/vec/exec/scan/vmeta_scanner.cpp b/be/src/vec/exec/scan/vmeta_scanner.cpp index 289930b16bc..db025672874 100644 --- a/be/src/vec/exec/scan/vmeta_scanner.cpp +++ b/be/src/vec/exec/scan/vmeta_scanner.cpp @@ -148,7 +148,7 @@ Status VMetaScanner::_fill_block_with_remote_data(const std::vector<MutableColum if (slot_desc->is_nullable()) { auto& null_col = reinterpret_cast<ColumnNullable&>(*col_ptr); null_col.get_null_map_data().push_back(0); - col_ptr = null_col.get_nested_column_ptr(); + col_ptr = null_col.get_nested_column_ptr().get(); } switch (slot_desc->type().type) { case TYPE_BOOLEAN: { diff --git a/be/src/vec/exprs/vcompound_pred.h b/be/src/vec/exprs/vcompound_pred.h index ff7649600b4..e3c02f554b3 100644 --- a/be/src/vec/exprs/vcompound_pred.h +++ b/be/src/vec/exprs/vcompound_pred.h @@ -272,8 +272,10 @@ public: auto col_res = ColumnUInt8::create(size); auto col_nulls = ColumnUInt8::create(size); - auto* __restrict res_datas = assert_cast<ColumnUInt8*>(col_res)->get_data().data(); - auto* __restrict res_nulls = assert_cast<ColumnUInt8*>(col_nulls)->get_data().data(); + auto* __restrict res_datas = + assert_cast<ColumnUInt8*>(col_res.get())->get_data().data(); + auto* __restrict res_nulls = + assert_cast<ColumnUInt8*>(col_nulls.get())->get_data().data(); ColumnPtr temp_null_map = nullptr; // maybe both children are nullable / or one of children is nullable auto* __restrict lhs_null_map_tmp = create_null_map_column(temp_null_map, lhs_null_map); diff --git a/be/src/vec/functions/array/function_array_cum_sum.cpp b/be/src/vec/functions/array/function_array_cum_sum.cpp index 2f93a2a83b1..5fba7d4a619 100644 --- a/be/src/vec/functions/array/function_array_cum_sum.cpp +++ b/be/src/vec/functions/array/function_array_cum_sum.cpp @@ -118,7 +118,7 @@ public: // get null map const ColumnNullable* src_nested_nullable_col = check_and_get_column<ColumnNullable>(*src_nested_column); - src_nested_column = src_nested_nullable_col->get_nested_column_ptr(); + src_nested_column = src_nested_nullable_col->get_nested_column_ptr().get(); const NullMapType& src_null_map = src_nested_nullable_col->get_null_map_column().get_data(); ColumnPtr res_nested_ptr; diff --git a/be/src/vec/functions/array/function_array_distinct.h b/be/src/vec/functions/array/function_array_distinct.h index 4b7e3e6f035..4d37f7cbcf7 100644 --- a/be/src/vec/functions/array/function_array_distinct.h +++ b/be/src/vec/functions/array/function_array_distinct.h @@ -102,14 +102,14 @@ public: if (src_nested_column->is_nullable()) { const auto* src_nested_nullable_col = check_and_get_column<ColumnNullable>(*src_nested_column); - src_nested_column = src_nested_nullable_col->get_nested_column_ptr(); + src_nested_column = src_nested_nullable_col->get_nested_column_ptr().get(); src_null_map = &src_nested_nullable_col->get_null_map_column().get_data(); } NullMapType* dest_null_map = nullptr; if (dest_nested_column->is_nullable()) { auto* dest_nested_nullable_col = reinterpret_cast<ColumnNullable*>(dest_nested_column); - dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr(); + dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr().get(); dest_null_map = &dest_nested_nullable_col->get_null_map_column().get_data(); } diff --git a/be/src/vec/functions/array/function_array_enumerate.cpp b/be/src/vec/functions/array/function_array_enumerate.cpp index 0e8bca3e5cd..3846addb83b 100644 --- a/be/src/vec/functions/array/function_array_enumerate.cpp +++ b/be/src/vec/functions/array/function_array_enumerate.cpp @@ -83,7 +83,7 @@ public: auto left_column = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); const ColumnArray* array = - check_and_get_column<ColumnArray>(remove_nullable(left_column->get_ptr())); + check_and_get_column<ColumnArray>(remove_nullable(left_column->get_ptr()).get()); if (!array) { return Status::RuntimeError( fmt::format("Illegal column {}, of first argument of function {}", @@ -107,7 +107,8 @@ public: ColumnPtr res_column = ColumnArray::create(std::move(nested_column), array->get_offsets_ptr()); if (block.get_by_position(arguments[0]).column->is_nullable()) { - const ColumnNullable* nullable = check_and_get_column<ColumnNullable>(left_column); + const ColumnNullable* nullable = + check_and_get_column<ColumnNullable>(left_column.get()); res_column = ColumnNullable::create( res_column, nullable->get_null_map_column().clone_resized(nullable->size())); } diff --git a/be/src/vec/functions/array/function_array_enumerate_uniq.cpp b/be/src/vec/functions/array/function_array_enumerate_uniq.cpp index 21d6ab40007..bdee406655f 100644 --- a/be/src/vec/functions/array/function_array_enumerate_uniq.cpp +++ b/be/src/vec/functions/array/function_array_enumerate_uniq.cpp @@ -128,7 +128,7 @@ public: block.get_by_position(arguments[i]).column->convert_to_full_column_if_const()); ColumnPtr& cur_column = src_columns[i]; const ColumnArray* array = - check_and_get_column<ColumnArray>(remove_nullable(cur_column->get_ptr())); + check_and_get_column<ColumnArray>(remove_nullable(cur_column->get_ptr()).get()); if (!array) { return Status::RuntimeError( fmt::format("Illegal column {}, of first argument of function {}", @@ -151,7 +151,7 @@ public: const NullMapType* null_map = nullptr; if (arguments.size() == 1 && data_columns[0]->is_nullable()) { const ColumnNullable* nullable = check_and_get_column<ColumnNullable>(*data_columns[0]); - data_columns[0] = nullable->get_nested_column_ptr(); + data_columns[0] = nullable->get_nested_column_ptr().get(); null_map = &nullable->get_null_map_column().get_data(); } @@ -219,7 +219,8 @@ public: if (arguments.size() == 1 && block.get_by_position(arguments[0]).column->is_nullable()) { auto left_column = block.get_by_position(arguments[0]).column->convert_to_full_column_if_const(); - const ColumnNullable* nullable = check_and_get_column<ColumnNullable>(left_column); + const ColumnNullable* nullable = + check_and_get_column<ColumnNullable>(left_column.get()); res_column = ColumnNullable::create( res_column, nullable->get_null_map_column().clone_resized(nullable->size())); } diff --git a/be/src/vec/functions/array/function_array_join.h b/be/src/vec/functions/array/function_array_join.h index 957b2288fb7..29521c36111 100644 --- a/be/src/vec/functions/array/function_array_join.h +++ b/be/src/vec/functions/array/function_array_join.h @@ -78,10 +78,11 @@ public: auto nested_type = data_type_array->get_nested_type(); auto dest_column_ptr = ColumnString::create(); - DCHECK(dest_column_ptr != nullptr); + DCHECK(dest_column_ptr); - auto res_val = _execute_by_type(*src.nested_col, *src.offsets_ptr, src.nested_nullmap_data, - sep_str, null_replace_str, nested_type, dest_column_ptr); + auto res_val = + _execute_by_type(*src.nested_col, *src.offsets_ptr, src.nested_nullmap_data, + sep_str, null_replace_str, nested_type, dest_column_ptr.get()); if (!res_val) { return Status::RuntimeError(fmt::format( "execute failed or unsupported types for function {}({},{},{})", "array_join", diff --git a/be/src/vec/functions/array/function_array_map.h b/be/src/vec/functions/array/function_array_map.h index fd4a2fc59f3..5bfe723e232 100644 --- a/be/src/vec/functions/array/function_array_map.h +++ b/be/src/vec/functions/array/function_array_map.h @@ -165,7 +165,7 @@ public: static Status execute(ColumnPtr& res_ptr, ColumnArrayExecutionDatas datas, std::vector<bool>& col_const, size_t start_row, size_t end_row) { ColumnArrayMutableData dst = - create_mutable_data(datas[0].nested_col, datas[0].nested_nullmap_data); + create_mutable_data(datas[0].nested_col.get(), datas[0].nested_nullmap_data); if (_execute_internal<ALL_COLUMNS_SIMPLE>(dst, datas, col_const, start_row, end_row)) { res_ptr = assemble_column_array(dst); return Status::OK(); diff --git a/be/src/vec/functions/array/function_array_pop.cpp b/be/src/vec/functions/array/function_array_pop.cpp index 2182699e020..1ddd767cfaf 100644 --- a/be/src/vec/functions/array/function_array_pop.cpp +++ b/be/src/vec/functions/array/function_array_pop.cpp @@ -75,7 +75,7 @@ public: } // prepare dst array column bool is_nullable = src.nested_nullmap_data != nullptr; - ColumnArrayMutableData dst = create_mutable_data(src.nested_col, is_nullable); + ColumnArrayMutableData dst = create_mutable_data(src.nested_col.get(), is_nullable); dst.offsets_ptr->reserve(input_rows_count); // start from index depending on the PopType::start_offset auto offset_column = ColumnInt64::create(array_column->size(), PopType::start_offset); diff --git a/be/src/vec/functions/array/function_array_range.cpp b/be/src/vec/functions/array/function_array_range.cpp index 8a3de375450..0980587660b 100644 --- a/be/src/vec/functions/array/function_array_range.cpp +++ b/be/src/vec/functions/array/function_array_range.cpp @@ -137,7 +137,7 @@ struct RangeImplUtil { IColumn* dest_nested_column = &dest_array_column_ptr->get_data(); ColumnNullable* dest_nested_nullable_col = reinterpret_cast<ColumnNullable*>(dest_nested_column); - dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr(); + dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr().get(); auto& dest_nested_null_map = dest_nested_nullable_col->get_null_map_column().get_data(); auto args_null_map = ColumnUInt8::create(input_rows_count, 0); diff --git a/be/src/vec/functions/array/function_array_remove.h b/be/src/vec/functions/array/function_array_remove.h index 197b032b0f8..661a18170ed 100644 --- a/be/src/vec/functions/array/function_array_remove.h +++ b/be/src/vec/functions/array/function_array_remove.h @@ -107,13 +107,13 @@ private: auto dst_nested_column = ColumnNullable::create(nested_column.clone_empty(), ColumnUInt8::create()); array_nested_column = dst_nested_column->get_ptr(); - dst_column = dst_nested_column->get_nested_column_ptr(); + dst_column = dst_nested_column->get_nested_column_ptr().get(); dst_null_map = &dst_nested_column->get_null_map_data(); dst_null_map->reserve(offsets.back()); } else { auto dst_nested_column = nested_column.clone_empty(); array_nested_column = dst_nested_column->get_ptr(); - dst_column = dst_nested_column; + dst_column = dst_nested_column.get(); } auto& dst_data = reinterpret_cast<NestedColumnType&>(*dst_column).get_data(); @@ -179,13 +179,13 @@ private: auto dst_nested_column = ColumnNullable::create(nested_column.clone_empty(), ColumnUInt8::create()); array_nested_column = dst_nested_column->get_ptr(); - dst_column = dst_nested_column->get_nested_column_ptr(); + dst_column = dst_nested_column->get_nested_column_ptr().get(); dst_null_map = &dst_nested_column->get_null_map_data(); dst_null_map->reserve(offsets.back()); } else { auto dst_nested_column = nested_column.clone_empty(); array_nested_column = dst_nested_column->get_ptr(); - dst_column = dst_nested_column; + dst_column = dst_nested_column.get(); } auto& dst_offs = reinterpret_cast<ColumnString&>(*dst_column).get_offsets(); diff --git a/be/src/vec/functions/array/function_array_reverse.h b/be/src/vec/functions/array/function_array_reverse.h index 8567bc61158..9fc16231518 100644 --- a/be/src/vec/functions/array/function_array_reverse.h +++ b/be/src/vec/functions/array/function_array_reverse.h @@ -40,7 +40,7 @@ struct ArrayReverseImpl { } bool is_nullable = src.nested_nullmap_data ? true : false; - ColumnArrayMutableData dst = create_mutable_data(src.nested_col, is_nullable); + ColumnArrayMutableData dst = create_mutable_data(src.nested_col.get(), is_nullable); dst.offsets_ptr->reserve(input_rows_count); auto res_val = _execute_internal(*src.nested_col, *src.offsets_ptr, *dst.nested_col, diff --git a/be/src/vec/functions/array/function_array_set.h b/be/src/vec/functions/array/function_array_set.h index 1ecf6d72531..975268b1e61 100644 --- a/be/src/vec/functions/array/function_array_set.h +++ b/be/src/vec/functions/array/function_array_set.h @@ -142,9 +142,9 @@ public: bool right_const) { ColumnArrayMutableData dst; if (left_data.nested_nullmap_data || right_data.nested_nullmap_data) { - dst = create_mutable_data(left_data.nested_col, true); + dst = create_mutable_data(left_data.nested_col.get(), true); } else { - dst = create_mutable_data(left_data.nested_col, false); + dst = create_mutable_data(left_data.nested_col.get(), false); } ColumnPtr res_column; if (left_const) { diff --git a/be/src/vec/functions/array/function_array_slice.h b/be/src/vec/functions/array/function_array_slice.h index 2acd1d3fbe1..76082b26602 100644 --- a/be/src/vec/functions/array/function_array_slice.h +++ b/be/src/vec/functions/array/function_array_slice.h @@ -89,7 +89,7 @@ public: } // prepare dst array column bool is_nullable = src.nested_nullmap_data ? true : false; - ColumnArrayMutableData dst = create_mutable_data(src.nested_col, is_nullable); + ColumnArrayMutableData dst = create_mutable_data(src.nested_col.get(), is_nullable); dst.offsets_ptr->reserve(input_rows_count); // execute slice_array(dst, src, *offset_column, length_column.get()); diff --git a/be/src/vec/functions/array/function_array_sortby.cpp b/be/src/vec/functions/array/function_array_sortby.cpp index 899bb40fba1..fe6799aaa2e 100644 --- a/be/src/vec/functions/array/function_array_sortby.cpp +++ b/be/src/vec/functions/array/function_array_sortby.cpp @@ -95,13 +95,13 @@ public: src_column_array.get_offsets_column().clone_resized(input_rows_count); MutableColumnPtr result_nullmap = nullptr; const ColumnUInt8::Container* src_null_map_data = nullptr; - if (argument_nullmap[0] != nullptr) { + if (argument_nullmap[0]) { const auto& src_column_nullmap = assert_cast<const ColumnUInt8&>(*argument_nullmap[0]); result_nullmap = src_column_nullmap.clone_resized(input_rows_count); src_null_map_data = &(src_column_nullmap.get_data()); } const ColumnUInt8::Container* key_null_map_data = nullptr; - if (argument_nullmap[1] != nullptr) { + if (argument_nullmap[1]) { const auto& key_column_nullmap = assert_cast<const ColumnUInt8&>(*argument_nullmap[1]); key_null_map_data = &(key_column_nullmap.get_data()); } @@ -149,7 +149,7 @@ public: } } src_nested_nullable_column.append_data_by_selector(result_data_column, src_selector); - if (result_nullmap != nullptr) { + if (result_nullmap) { block.replace_by_position( result, ColumnNullable::create(ColumnArray::create(std::move(result_data_column), diff --git a/be/src/vec/functions/array/function_arrays_overlap.h b/be/src/vec/functions/array/function_arrays_overlap.h index dd993100885..8ac21bcd710 100644 --- a/be/src/vec/functions/array/function_arrays_overlap.h +++ b/be/src/vec/functions/array/function_arrays_overlap.h @@ -370,11 +370,11 @@ private: ExecutorImpl impl; if (right_size < left_size) { - impl.insert_array(right_data.nested_col, right_start, right_size); - dst_data[row] = impl.find_any(left_data.nested_col, left_start, left_size); + impl.insert_array(right_data.nested_col.get(), right_start, right_size); + dst_data[row] = impl.find_any(left_data.nested_col.get(), left_start, left_size); } else { - impl.insert_array(left_data.nested_col, left_start, left_size); - dst_data[row] = impl.find_any(right_data.nested_col, right_start, right_size); + impl.insert_array(left_data.nested_col.get(), left_start, left_size); + dst_data[row] = impl.find_any(right_data.nested_col.get(), right_start, right_size); } } return Status::OK(); diff --git a/be/src/vec/functions/comparison_equal_for_null.cpp b/be/src/vec/functions/comparison_equal_for_null.cpp index 919f9ebed65..35719cf5730 100644 --- a/be/src/vec/functions/comparison_equal_for_null.cpp +++ b/be/src/vec/functions/comparison_equal_for_null.cpp @@ -139,18 +139,20 @@ public: left_column = check_and_get_column<const ColumnNullable>( assert_cast<const ColumnConst*, TypeCheckOnRelease::DISABLE>( col_left.column.get()) - ->get_data_column_ptr()); + ->get_data_column_ptr() + .get()); } else { - left_column = check_and_get_column<const ColumnNullable>(col_left.column); + left_column = check_and_get_column<const ColumnNullable>(col_left.column.get()); } if (right_const) { right_column = check_and_get_column<const ColumnNullable>( assert_cast<const ColumnConst*, TypeCheckOnRelease::DISABLE>( col_right.column.get()) - ->get_data_column_ptr()); + ->get_data_column_ptr() + .get()); } else { - right_column = check_and_get_column<const ColumnNullable>(col_right.column); + right_column = check_and_get_column<const ColumnNullable>(col_right.column.get()); } bool left_nullable = left_column != nullptr; diff --git a/be/src/vec/functions/function_agg_state.h b/be/src/vec/functions/function_agg_state.h index f4b7aef23af..84a8d4f6f8b 100644 --- a/be/src/vec/functions/function_agg_state.h +++ b/be/src/vec/functions/function_agg_state.h @@ -82,7 +82,7 @@ public: save_columns.push_back(column); } - agg_columns.push_back(column); + agg_columns.push_back(column.get()); } _agg_function->streaming_agg_serialize_to_column(agg_columns.data(), col, input_rows_count, &(context->get_arena())); diff --git a/be/src/vec/functions/function_binary_arithmetic.h b/be/src/vec/functions/function_binary_arithmetic.h index 4c0b8e7a089..a2757b38346 100644 --- a/be/src/vec/functions/function_binary_arithmetic.h +++ b/be/src/vec/functions/function_binary_arithmetic.h @@ -165,7 +165,7 @@ struct BinaryOperationImpl { static ColumnPtr adapt_normal_vector_constant(ColumnPtr column_left, B b) { auto column_left_ptr = - check_and_get_column<typename Base::Traits::ColumnVectorA>(column_left); + check_and_get_column<typename Base::Traits::ColumnVectorA>(column_left.get()); auto column_result = Base::ColumnVectorResult::create(column_left->size()); DCHECK(column_left_ptr != nullptr); @@ -182,7 +182,7 @@ struct BinaryOperationImpl { static ColumnPtr adapt_normal_constant_vector(A a, ColumnPtr column_right) { auto column_right_ptr = - check_and_get_column<typename Base::Traits::ColumnVectorB>(column_right); + check_and_get_column<typename Base::Traits::ColumnVectorB>(column_right.get()); auto column_result = Base::ColumnVectorResult::create(column_right->size()); DCHECK(column_right_ptr != nullptr); @@ -199,9 +199,9 @@ struct BinaryOperationImpl { static ColumnPtr adapt_normal_vector_vector(ColumnPtr column_left, ColumnPtr column_right) { auto column_left_ptr = - check_and_get_column<typename Base::Traits::ColumnVectorA>(column_left); + check_and_get_column<typename Base::Traits::ColumnVectorA>(column_left.get()); auto column_right_ptr = - check_and_get_column<typename Base::Traits::ColumnVectorB>(column_right); + check_and_get_column<typename Base::Traits::ColumnVectorB>(column_right.get()); auto column_result = Base::ColumnVectorResult::create(column_left->size()); DCHECK(column_left_ptr != nullptr && column_right_ptr != nullptr); @@ -447,7 +447,8 @@ public: auto type_result = assert_cast<const DataTypeDecimal<ResultType>&, TypeCheckOnRelease::DISABLE>( *res_data_type); - auto column_left_ptr = check_and_get_column<typename Traits::ColumnVectorA>(column_left); + auto column_left_ptr = + check_and_get_column<typename Traits::ColumnVectorA>(column_left.get()); auto column_result = ColumnDecimal<ResultType>::create( column_left->size(), assert_cast<const DataTypeDecimal<ResultType>&, TypeCheckOnRelease::DISABLE>( @@ -482,7 +483,8 @@ public: auto type_result = assert_cast<const DataTypeDecimal<ResultType>&, TypeCheckOnRelease::DISABLE>( *res_data_type); - auto column_right_ptr = check_and_get_column<typename Traits::ColumnVectorB>(column_right); + auto column_right_ptr = + check_and_get_column<typename Traits::ColumnVectorB>(column_right.get()); auto column_result = ColumnDecimal<ResultType>::create( column_right->size(), assert_cast<const DataTypeDecimal<ResultType>&, TypeCheckOnRelease::DISABLE>( @@ -515,8 +517,10 @@ public: const ResultType& max_result_number, const ResultType& scale_diff_multiplier, DataTypePtr res_data_type) { - auto column_left_ptr = check_and_get_column<typename Traits::ColumnVectorA>(column_left); - auto column_right_ptr = check_and_get_column<typename Traits::ColumnVectorB>(column_right); + auto column_left_ptr = + check_and_get_column<typename Traits::ColumnVectorA>(column_left.get()); + auto column_right_ptr = + check_and_get_column<typename Traits::ColumnVectorB>(column_right.get()); const auto& type_result = assert_cast<const DataTypeDecimal<ResultType>&>(*res_data_type); auto column_result = @@ -847,8 +851,8 @@ private: static ColumnPtr constant_constant(ColumnPtr column_left, ColumnPtr column_right, const LeftDataType& type_left, const RightDataType& type_right, DataTypePtr res_data_type) { - auto column_left_ptr = check_and_get_column<ColumnConst>(column_left); - auto column_right_ptr = check_and_get_column<ColumnConst>(column_right); + const auto* column_left_ptr = check_and_get_column<ColumnConst>(column_left.get()); + const auto* column_right_ptr = check_and_get_column<ColumnConst>(column_right.get()); DCHECK(column_left_ptr != nullptr && column_right_ptr != nullptr); ColumnPtr column_result = nullptr; @@ -875,7 +879,7 @@ private: static ColumnPtr vector_constant(ColumnPtr column_left, ColumnPtr column_right, const LeftDataType& type_left, const RightDataType& type_right, DataTypePtr res_data_type) { - auto column_right_ptr = check_and_get_column<ColumnConst>(column_right); + const auto* column_right_ptr = check_and_get_column<ColumnConst>(column_right.get()); DCHECK(column_right_ptr != nullptr); if constexpr (result_is_decimal) { @@ -894,7 +898,7 @@ private: static ColumnPtr constant_vector(ColumnPtr column_left, ColumnPtr column_right, const LeftDataType& type_left, const RightDataType& type_right, DataTypePtr res_data_type) { - auto column_left_ptr = check_and_get_column<ColumnConst>(column_left); + const auto* column_left_ptr = check_and_get_column<ColumnConst>(column_left.get()); DCHECK(column_left_ptr != nullptr); if constexpr (result_is_decimal) { diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp index 92a5dba7b7a..96cae50a9ba 100644 --- a/be/src/vec/functions/function_bitmap.cpp +++ b/be/src/vec/functions/function_bitmap.cpp @@ -1211,7 +1211,7 @@ public: IColumn* dest_nested_column = &dest_array_column_ptr->get_data(); ColumnNullable* dest_nested_nullable_col = reinterpret_cast<ColumnNullable*>(dest_nested_column); - dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr(); + dest_nested_column = dest_nested_nullable_col->get_nested_column_ptr().get(); auto& dest_nested_null_map = dest_nested_nullable_col->get_null_map_column().get_data(); auto& arg_col = block.get_by_position(arguments[0]).column; diff --git a/be/src/vec/functions/function_bitmap_variadic.cpp b/be/src/vec/functions/function_bitmap_variadic.cpp index 6e1a103fdbd..47a159e3c2f 100644 --- a/be/src/vec/functions/function_bitmap_variadic.cpp +++ b/be/src/vec/functions/function_bitmap_variadic.cpp @@ -247,7 +247,7 @@ public: vec_res.resize(input_rows_count); RETURN_IF_ERROR(Impl::vector_vector(argument_columns.data(), argument_size, - input_rows_count, vec_res, col_res_nulls)); + input_rows_count, vec_res, col_res_nulls.get())); if (!use_default_implementation_for_nulls() && result_info.type->is_nullable()) { block.replace_by_position( result, ColumnNullable::create(std::move(col_res), std::move(col_res_nulls))); diff --git a/be/src/vec/functions/function_case.h b/be/src/vec/functions/function_case.h index af44ea0d9b1..81f08f682ef 100644 --- a/be/src/vec/functions/function_case.h +++ b/be/src/vec/functions/function_case.h @@ -318,7 +318,7 @@ public: const uint8* __restrict then_idx, CaseWhenColumnHolder& column_holder) const { for (auto& then_ptr : column_holder.then_ptrs) { - then_ptr->reset(then_ptr.value()->convert_to_full_column_if_const()); + then_ptr->reset(then_ptr.value()->convert_to_full_column_if_const().get()); } size_t rows_count = column_holder.rows_count; diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index af9e9d19267..483e837de5d 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -770,7 +770,7 @@ struct ConvertImplGenericFromJsonb { continue; } ReadBuffer read_buffer((char*)(input_str.data()), input_str.size()); - Status st = data_type_to->from_string(read_buffer, col_to); + Status st = data_type_to->from_string(read_buffer, col_to.get()); // if parsing failed, will return null (*vec_null_map_to)[i] = !st.ok(); if (!st.ok()) { diff --git a/be/src/vec/functions/function_collection_in.h b/be/src/vec/functions/function_collection_in.h index ce58d63f44b..35299c7ea67 100644 --- a/be/src/vec/functions/function_collection_in.h +++ b/be/src/vec/functions/function_collection_in.h @@ -117,7 +117,8 @@ public: DCHECK(const_column_ptr != nullptr); const auto& [col, _] = unpack_if_const(const_column_ptr->column_ptr); if (col->is_nullable()) { - auto* null_col = vectorized::check_and_get_column<vectorized::ColumnNullable>(col); + const auto* null_col = + vectorized::check_and_get_column<vectorized::ColumnNullable>(col.get()); if (null_col->has_null()) { state->null_in_set = true; } else { @@ -161,7 +162,7 @@ public: if (materialized_column_not_null->is_nullable()) { materialized_column_not_null = assert_cast<ColumnPtr>( vectorized::check_and_get_column<vectorized::ColumnNullable>( - materialized_column_not_null) + materialized_column_not_null.get()) ->get_nested_column_ptr()); } diff --git a/be/src/vec/functions/function_date_or_datetime_computation.h b/be/src/vec/functions/function_date_or_datetime_computation.h index 330ea75cba9..224bf491791 100644 --- a/be/src/vec/functions/function_date_or_datetime_computation.h +++ b/be/src/vec/functions/function_date_or_datetime_computation.h @@ -878,7 +878,7 @@ struct CurrentDateTimeImpl { bool use_const; if constexpr (WithPrecision) { if (const auto* const_column = check_and_get_column<ColumnConst>( - block.get_by_position(arguments[0]).column)) { + block.get_by_position(arguments[0]).column.get())) { int64_t scale = const_column->get_int(0); dtv.from_unixtime(context->state()->timestamp_ms() / 1000, context->state()->nano_seconds(), @@ -892,7 +892,7 @@ struct CurrentDateTimeImpl { use_const = true; } else if (const auto* nullable_column = check_and_get_column<ColumnNullable>( - block.get_by_position(arguments[0]).column)) { + block.get_by_position(arguments[0]).column.get())) { const auto& null_map = nullable_column->get_null_map_data(); const auto& nested_column = assert_cast<const ColumnInt32*>( nullable_column->get_nested_column_ptr().get()); diff --git a/be/src/vec/functions/function_ip.h b/be/src/vec/functions/function_ip.h index 1a1c23e2b06..9f2f4dc2886 100644 --- a/be/src/vec/functions/function_ip.h +++ b/be/src/vec/functions/function_ip.h @@ -768,11 +768,13 @@ public: if (is_ipv4(addr_column_with_type_and_name.type)) { execute_impl_with_ip<PrimitiveType::TYPE_IPV4, ColumnIPv4>( input_rows_count, addr_const, cidr_const, - assert_cast<const ColumnString*>(cidr_column.get()), addr_column, col_res); + assert_cast<const ColumnString*>(cidr_column.get()), addr_column, + col_res.get()); } else if (is_ipv6(addr_column_with_type_and_name.type)) { execute_impl_with_ip<PrimitiveType::TYPE_IPV6, ColumnIPv6>( input_rows_count, addr_const, cidr_const, - assert_cast<const ColumnString*>(cidr_column.get()), addr_column, col_res); + assert_cast<const ColumnString*>(cidr_column.get()), addr_column, + col_res.get()); } else { const auto* str_addr_column = assert_cast<const ColumnString*>(addr_column.get()); const auto* str_cidr_column = assert_cast<const ColumnString*>(cidr_column.get()); diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp index 463508169aa..dcae26f3c2f 100644 --- a/be/src/vec/functions/function_jsonb.cpp +++ b/be/src/vec/functions/function_jsonb.cpp @@ -459,11 +459,12 @@ public: // prepare jsonb data column jsonb_data_column = unpack_if_const(block.get_by_position(arguments[0]).column).first; if (block.get_by_position(arguments[0]).column->is_nullable()) { - const auto* nullable = check_and_get_column<ColumnNullable>(jsonb_data_column); + const auto* nullable = check_and_get_column<ColumnNullable>(jsonb_data_column.get()); jsonb_data_column = nullable->get_nested_column_ptr(); data_null_map = &nullable->get_null_map_data(); } - const ColumnString* col_from_string = check_and_get_column<ColumnString>(jsonb_data_column); + const ColumnString* col_from_string = + check_and_get_column<ColumnString>(jsonb_data_column.get()); // prepare parse path column prepare, maybe we do not have path column ColumnPtr jsonb_path_column = nullptr; @@ -475,11 +476,12 @@ public: std::tie(jsonb_path_column, path_const) = unpack_if_const(block.get_by_position(arguments[1]).column); if (block.get_by_position(arguments[1]).column->is_nullable()) { - const auto* nullable = check_and_get_column<ColumnNullable>(jsonb_path_column); + const auto* nullable = + check_and_get_column<ColumnNullable>(jsonb_path_column.get()); jsonb_path_column = nullable->get_nested_column_ptr(); path_null_map = &nullable->get_null_map_data(); } - jsonb_path_col = check_and_get_column<ColumnString>(jsonb_path_column); + jsonb_path_col = check_and_get_column<ColumnString>(jsonb_path_column.get()); } auto null_map = ColumnUInt8::create(input_rows_count, 0); @@ -1844,9 +1846,10 @@ public: // prepare jsonb data column std::tie(col_json, json_is_const) = unpack_if_const(block.get_by_position(arguments[0]).column); - const ColumnString* col_json_string = check_and_get_column<ColumnString>(col_json); - if (auto* nullable = check_and_get_column<ColumnNullable>(col_json)) { - col_json_string = check_and_get_column<ColumnString>(nullable->get_nested_column_ptr()); + const ColumnString* col_json_string = check_and_get_column<ColumnString>(col_json.get()); + if (auto* nullable = check_and_get_column<ColumnNullable>(col_json.get())) { + col_json_string = + check_and_get_column<ColumnString>(nullable->get_nested_column_ptr().get()); } if (!col_json_string) { @@ -1873,8 +1876,8 @@ public: // prepare jsonb data column std::tie(col_one, one_is_const) = unpack_if_const(block.get_by_position(arguments[1]).column); - const ColumnString* col_one_string = check_and_get_column<ColumnString>(col_one); - if (auto* nullable = check_and_get_column<ColumnNullable>(col_one)) { + const ColumnString* col_one_string = check_and_get_column<ColumnString>(col_one.get()); + if (auto* nullable = check_and_get_column<ColumnNullable>(col_one.get())) { col_one_string = check_and_get_column<ColumnString>(*nullable->get_nested_column_ptr()); } if (!col_one_string) { @@ -1921,8 +1924,9 @@ public: std::tie(col_search, search_is_const) = unpack_if_const(block.get_by_position(arguments[2]).column); - const ColumnString* col_search_string = check_and_get_column<ColumnString>(col_search); - if (auto* nullable = check_and_get_column<ColumnNullable>(col_search)) { + const ColumnString* col_search_string = + check_and_get_column<ColumnString>(col_search.get()); + if (auto* nullable = check_and_get_column<ColumnNullable>(col_search.get())) { col_search_string = check_and_get_column<ColumnString>(*nullable->get_nested_column_ptr()); } diff --git a/be/src/vec/functions/function_nullables.cpp b/be/src/vec/functions/function_nullables.cpp index 91bce24f48f..b1e72ff52a7 100644 --- a/be/src/vec/functions/function_nullables.cpp +++ b/be/src/vec/functions/function_nullables.cpp @@ -54,7 +54,8 @@ public: Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, uint32_t result, size_t input_rows_count) const override { ColumnPtr& col = block.get_by_position(arguments[0]).column; - if (const auto* col_null = check_and_get_column<ColumnNullable>(col); col_null == nullptr) { + if (const auto* col_null = check_and_get_column<ColumnNullable>(col.get()); + col_null == nullptr) { // not null block.replace_by_position( result, ColumnNullable::create(col, ColumnBool::create(input_rows_count, 0))); @@ -85,7 +86,7 @@ public: Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, uint32_t result, size_t input_rows_count) const override { auto& data = block.get_by_position(arguments[0]); - if (const auto* col_null = check_and_get_column<ColumnNullable>(data.column); + if (const auto* col_null = check_and_get_column<ColumnNullable>(data.column.get()); col_null == nullptr) // raise error if input is not nullable. { return Status::InvalidArgument( diff --git a/be/src/vec/functions/function_quantile_state.cpp b/be/src/vec/functions/function_quantile_state.cpp index 95afbf1db32..8f8740841c5 100644 --- a/be/src/vec/functions/function_quantile_state.cpp +++ b/be/src/vec/functions/function_quantile_state.cpp @@ -130,7 +130,7 @@ public: const ColumnPtr& column = block.get_by_position(arguments[0]).column; const DataTypePtr& data_type = block.get_by_position(arguments[0]).type; auto compression_arg = check_and_get_column_const<ColumnFloat32>( - block.get_by_position(arguments.back()).column); + block.get_by_position(arguments.back()).column.get()); float compression = 2048; if (compression_arg) { auto compression_arg_val = compression_arg->get_value<Float32>(); @@ -189,7 +189,7 @@ public: auto str_col = assert_cast<const ColumnQuantileState*>(column.get()); auto& col_data = str_col->get_data(); auto percent_arg = check_and_get_column_const<ColumnFloat32>( - block.get_by_position(arguments.back()).column); + block.get_by_position(arguments.back()).column.get()); if (!percent_arg) { return Status::InternalError( diff --git a/be/src/vec/functions/function_string.h b/be/src/vec/functions/function_string.h index 14926e1062c..a729af5948a 100644 --- a/be/src/vec/functions/function_string.h +++ b/be/src/vec/functions/function_string.h @@ -2142,7 +2142,7 @@ public: NullMapType* dest_nested_null_map = nullptr; auto* dest_nullable_col = reinterpret_cast<ColumnNullable*>(dest_nested_column); - dest_nested_column = dest_nullable_col->get_nested_column_ptr(); + dest_nested_column = dest_nullable_col->get_nested_column_ptr().get(); dest_nested_null_map = &dest_nullable_col->get_null_map_column().get_data(); const auto* col_left = check_and_get_column<ColumnString>(src_column.get()); @@ -4436,7 +4436,7 @@ public: } else if (is_ascii) { impl_vectors = impl_vectors_ascii<false>; } - impl_vectors(col_source, col_from, col_to, col_res); + impl_vectors(col_source, col_from, col_to, col_res.get()); block.get_by_position(result).column = std::move(col_res); return Status::OK(); } diff --git a/be/src/vec/functions/function_tokenize.cpp b/be/src/vec/functions/function_tokenize.cpp index 0bcd31af40d..f0a7c3b68ae 100644 --- a/be/src/vec/functions/function_tokenize.cpp +++ b/be/src/vec/functions/function_tokenize.cpp @@ -129,7 +129,7 @@ Status FunctionTokenize::execute_impl(FunctionContext* /*context*/, Block& block NullMapType* dest_nested_null_map = nullptr; ColumnNullable* dest_nullable_col = reinterpret_cast<ColumnNullable*>(dest_nested_column); - dest_nested_column = dest_nullable_col->get_nested_column_ptr(); + dest_nested_column = dest_nullable_col->get_nested_column_ptr().get(); dest_nested_null_map = &dest_nullable_col->get_null_map_column().get_data(); if (auto col_left = check_and_get_column<ColumnString>(src_column.get())) { diff --git a/be/src/vec/functions/functions_geo.cpp b/be/src/vec/functions/functions_geo.cpp index 6d75258d146..0a752af18fe 100644 --- a/be/src/vec/functions/functions_geo.cpp +++ b/be/src/vec/functions/functions_geo.cpp @@ -258,10 +258,10 @@ struct StDistanceSphere { ColumnPtr y_lat_origin = block.get_by_position(arguments[3]).column->convert_to_full_column_if_const(); - const auto* x_lng = check_and_get_column<ColumnFloat64>(x_lng_origin); - const auto* x_lat = check_and_get_column<ColumnFloat64>(x_lat_origin); - const auto* y_lng = check_and_get_column<ColumnFloat64>(y_lng_origin); - const auto* y_lat = check_and_get_column<ColumnFloat64>(y_lat_origin); + const auto* x_lng = check_and_get_column<ColumnFloat64>(x_lng_origin.get()); + const auto* x_lat = check_and_get_column<ColumnFloat64>(x_lat_origin.get()); + const auto* y_lng = check_and_get_column<ColumnFloat64>(y_lng_origin.get()); + const auto* y_lat = check_and_get_column<ColumnFloat64>(y_lat_origin.get()); CHECK(x_lng && x_lat && y_lng && y_lat); const auto size = x_lng->size(); @@ -305,10 +305,10 @@ struct StAngleSphere { ColumnPtr y_lat_origin = block.get_by_position(arguments[3]).column->convert_to_full_column_if_const(); - const auto* x_lng = check_and_get_column<ColumnFloat64>(x_lng_origin); - const auto* x_lat = check_and_get_column<ColumnFloat64>(x_lat_origin); - const auto* y_lng = check_and_get_column<ColumnFloat64>(y_lng_origin); - const auto* y_lat = check_and_get_column<ColumnFloat64>(y_lat_origin); + const auto* x_lng = check_and_get_column<ColumnFloat64>(x_lng_origin.get()); + const auto* x_lat = check_and_get_column<ColumnFloat64>(x_lat_origin.get()); + const auto* y_lng = check_and_get_column<ColumnFloat64>(y_lng_origin.get()); + const auto* y_lat = check_and_get_column<ColumnFloat64>(y_lat_origin.get()); CHECK(x_lng && x_lat && y_lng && y_lat); const auto size = x_lng->size(); diff --git a/be/src/vec/functions/functions_logical.cpp b/be/src/vec/functions/functions_logical.cpp index 0f474851f03..f99f0447725 100644 --- a/be/src/vec/functions/functions_logical.cpp +++ b/be/src/vec/functions/functions_logical.cpp @@ -141,11 +141,11 @@ void basic_execute_impl(ColumnRawPtrs arguments, ColumnWithTypeAndName& result_i size_t input_rows_count) { auto col_res = ColumnUInt8::create(input_rows_count); if (auto l = check_and_get_column<ColumnConst>(arguments[0])) { - vector_const<Op>(arguments[1], l, col_res, input_rows_count); + vector_const<Op>(arguments[1], l, col_res.get(), input_rows_count); } else if (auto r = check_and_get_column<ColumnConst>(arguments[1])) { - vector_const<Op>(arguments[0], r, col_res, input_rows_count); + vector_const<Op>(arguments[0], r, col_res.get(), input_rows_count); } else { - vector_vector<Op>(arguments[0], arguments[1], col_res, input_rows_count); + vector_vector<Op>(arguments[0], arguments[1], col_res.get(), input_rows_count); } result_info.column = std::move(col_res); } @@ -156,11 +156,12 @@ void null_execute_impl(ColumnRawPtrs arguments, ColumnWithTypeAndName& result_in auto col_nulls = ColumnUInt8::create(input_rows_count); auto col_res = ColumnUInt8::create(input_rows_count); if (auto l = check_and_get_column<ColumnConst>(arguments[0])) { - vector_const_null<Op>(arguments[1], l, col_res, col_nulls, input_rows_count); + vector_const_null<Op>(arguments[1], l, col_res.get(), col_nulls.get(), input_rows_count); } else if (auto r = check_and_get_column<ColumnConst>(arguments[1])) { - vector_const_null<Op>(arguments[0], r, col_res, col_nulls, input_rows_count); + vector_const_null<Op>(arguments[0], r, col_res.get(), col_nulls.get(), input_rows_count); } else { - vector_vector_null<Op>(arguments[0], arguments[1], col_res, col_nulls, input_rows_count); + vector_vector_null<Op>(arguments[0], arguments[1], col_res.get(), col_nulls.get(), + input_rows_count); } result_info.column = ColumnNullable::create(std::move(col_res), std::move(col_nulls)); } diff --git a/be/src/vec/functions/in.h b/be/src/vec/functions/in.h index 5d590190182..6f697ba7441 100644 --- a/be/src/vec/functions/in.h +++ b/be/src/vec/functions/in.h @@ -216,7 +216,7 @@ public: if (materialized_column->is_nullable()) { const auto* null_col_ptr = vectorized::check_and_get_column<vectorized::ColumnNullable>( - materialized_column); + materialized_column.get()); const auto& null_map = assert_cast<const vectorized::ColumnUInt8&>( null_col_ptr->get_null_map_column()) .get_data(); diff --git a/be/src/vec/functions/least_greast.cpp b/be/src/vec/functions/least_greast.cpp index 7d1953f7041..9ad53c4f531 100644 --- a/be/src/vec/functions/least_greast.cpp +++ b/be/src/vec/functions/least_greast.cpp @@ -173,7 +173,7 @@ struct FunctionFieldImpl { size_t input_rows_count) { const auto& data_type = block.get_by_position(arguments[0]).type; auto result_column = ColumnInt32::create(input_rows_count, 0); - auto& res_data = static_cast<ColumnInt32*>(result_column)->get_data(); + auto& res_data = static_cast<ColumnInt32*>(result_column.get())->get_data(); const auto& column_size = arguments.size(); std::vector<ColumnPtr> argument_columns(column_size); diff --git a/be/src/vec/functions/round.h b/be/src/vec/functions/round.h index 3f4f9c60fcb..3b821f0aa52 100644 --- a/be/src/vec/functions/round.h +++ b/be/src/vec/functions/round.h @@ -731,6 +731,7 @@ public: const auto* col_general = is_col_general_const ? assert_cast<const ColumnConst&>(*column_general.column) .get_data_column_ptr() + .get() : column_general.column.get(); ColumnPtr res; diff --git a/be/src/vec/sink/vtablet_block_convertor.cpp b/be/src/vec/sink/vtablet_block_convertor.cpp index 26de6ea6c7e..466902a4f90 100644 --- a/be/src/vec/sink/vtablet_block_convertor.cpp +++ b/be/src/vec/sink/vtablet_block_convertor.cpp @@ -506,7 +506,8 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si vectorized::ColumnInt64::Container& dst_values = dst_column->get_data(); vectorized::ColumnPtr src_column_ptr = block->get_by_position(idx).column; - if (const auto* const_column = check_and_get_column<vectorized::ColumnConst>(src_column_ptr)) { + if (const auto* const_column = + check_and_get_column<vectorized::ColumnConst>(src_column_ptr.get())) { // for insert stmt like "insert into tbl1 select null,col1,col2,... from tbl2" or // "insert into tbl1 select 1,col1,col2,... from tbl2", the type of literal's column // will be `ColumnConst` @@ -530,7 +531,7 @@ Status OlapTableBlockConvertor::_fill_auto_inc_cols(vectorized::Block* block, si dst_values.resize_fill(rows, value); } } else if (const auto* src_nullable_column = - check_and_get_column<vectorized::ColumnNullable>(src_column_ptr)) { + check_and_get_column<vectorized::ColumnNullable>(src_column_ptr.get())) { auto src_nested_column_ptr = src_nullable_column->get_nested_column_ptr(); const auto& null_map_data = src_nullable_column->get_null_map_data(); dst_values.reserve(rows); diff --git a/be/src/vec/sink/writer/iceberg/partition_transformers.h b/be/src/vec/sink/writer/iceberg/partition_transformers.h index 79eb385b298..0b18ce24952 100644 --- a/be/src/vec/sink/writer/iceberg/partition_transformers.h +++ b/be/src/vec/sink/writer/iceberg/partition_transformers.h @@ -153,8 +153,8 @@ public: ColumnPtr string_column_ptr; ColumnPtr null_map_column_ptr; bool is_nullable = false; - if (auto* nullable_column = - check_and_get_column<ColumnNullable>(column_with_type_and_name.column)) { + if (const auto* nullable_column = + check_and_get_column<ColumnNullable>(column_with_type_and_name.column.get())) { null_map_column_ptr = nullable_column->get_null_map_column_ptr(); string_column_ptr = nullable_column->get_nested_column_ptr(); is_nullable = true; @@ -211,7 +211,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -270,7 +270,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -332,8 +332,8 @@ public: ColumnPtr column_ptr; ColumnPtr null_map_column_ptr; bool is_nullable = false; - if (auto* nullable_column = - check_and_get_column<ColumnNullable>(column_with_type_and_name.column)) { + if (const auto* nullable_column = + check_and_get_column<ColumnNullable>(column_with_type_and_name.column.get())) { null_map_column_ptr = nullable_column->get_null_map_column_ptr(); column_ptr = nullable_column->get_nested_column_ptr(); is_nullable = true; @@ -342,7 +342,7 @@ public: is_nullable = false; } - const auto* const decimal_col = check_and_get_column<ColumnDecimal<T>>(column_ptr); + const auto* const decimal_col = check_and_get_column<ColumnDecimal<T>>(column_ptr.get()); const auto& vec_src = decimal_col->get_data(); auto col_res = ColumnDecimal<T>::create(vec_src.size(), decimal_col->get_scale()); @@ -391,7 +391,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -454,7 +454,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -518,7 +518,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -597,7 +597,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -665,7 +665,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -746,7 +746,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -811,7 +811,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -883,7 +883,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -955,7 +955,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -1027,7 +1027,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -1099,7 +1099,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -1177,7 +1177,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -1254,7 +1254,7 @@ public: //1) get the target column ptr const ColumnWithTypeAndName& column_with_type_and_name = block.get_by_position(column_pos); ColumnPtr column_ptr = column_with_type_and_name.column->convert_to_full_column_if_const(); - CHECK(column_ptr != nullptr); + CHECK(column_ptr); //2) get the input data from block ColumnPtr null_map_column_ptr; @@ -1328,7 +1328,7 @@ public: ColumnPtr column_ptr; ColumnPtr null_map_column_ptr; if (auto* nullable_column = - check_and_get_column<ColumnNullable>(column_with_type_and_name.column)) { + check_and_get_column<ColumnNullable>(column_with_type_and_name.column.get())) { null_map_column_ptr = nullable_column->get_null_map_column_ptr(); column_ptr = nullable_column->get_nested_column_ptr(); } else { diff --git a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp index 29c97b59ea4..608afced8d9 100644 --- a/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp +++ b/be/src/vec/sink/writer/iceberg/viceberg_table_writer.cpp @@ -410,7 +410,7 @@ std::any VIcebergTableWriter::_get_iceberg_partition_value( int position) { //1) get the partition column ptr ColumnPtr col_ptr = partition_column.column->convert_to_full_column_if_const(); - CHECK(col_ptr != nullptr); + CHECK(col_ptr); if (col_ptr->is_nullable()) { const ColumnNullable* nullable_column = reinterpret_cast<const vectorized::ColumnNullable*>(col_ptr.get()); diff --git a/be/src/vec/utils/util.hpp b/be/src/vec/utils/util.hpp index 8d17b2787a5..485d81311ba 100644 --- a/be/src/vec/utils/util.hpp +++ b/be/src/vec/utils/util.hpp @@ -197,7 +197,7 @@ inline void change_null_to_true(ColumnPtr column, ColumnPtr argument = nullptr) data[i] |= null_map[i]; } memset(null_map, 0, rows); - } else if (argument != nullptr && argument->has_null()) { + } else if (argument && argument->has_null()) { const auto* __restrict null_map = assert_cast<const ColumnNullable*>(argument.get())->get_null_map_data().data(); auto* __restrict data = diff --git a/be/test/vec/columns/common_column_test.h b/be/test/vec/columns/common_column_test.h index 8e1b86c0168..b70ac660136 100644 --- a/be/test/vec/columns/common_column_test.h +++ b/be/test/vec/columns/common_column_test.h @@ -989,7 +989,7 @@ public: // check size EXPECT_EQ(ptr->size(), *cl); // check ptr is not the same - EXPECT_NE(ptr.get(), source_column); + EXPECT_NE(ptr.get(), source_column.get()); // check after clone_resized with assert_res auto ser_col = ColumnString::create(); @@ -1042,7 +1042,7 @@ public: // check size EXPECT_EQ(ptr->size(), insert_size); // check ptr is not the same - EXPECT_NE(ptr.get(), source_column); + EXPECT_NE(ptr.get(), source_column.get()); // check after cut with assert_res auto ser_col = ColumnString::create(); ser_col->reserve(ptr->size()); @@ -1095,7 +1095,7 @@ public: // check size EXPECT_EQ(ptr->size(), insert_size); // check ptr is not the same - EXPECT_NE(ptr.get(), source_column); + EXPECT_NE(ptr.get(), source_column.get()); // check after cut with assert_res auto ser_col = ColumnString::create(); ser_col->reserve(ptr->size()); diff --git a/be/test/vec/data_types/from_string_test.cpp b/be/test/vec/data_types/from_string_test.cpp index 01515b805d9..eb8b00ab16f 100644 --- a/be/test/vec/data_types/from_string_test.cpp +++ b/be/test/vec/data_types/from_string_test.cpp @@ -203,7 +203,7 @@ TEST(FromStringTest, ScalaWrapperFieldVsDataType) { string test_str = std::get<1>(type_pair)[i]; // data_type from_string ReadBuffer rb_test(test_str.data(), test_str.size()); - Status st = data_type_ptr->from_string(rb_test, col); + Status st = data_type_ptr->from_string(rb_test, col.get()); if (std::get<3>(type_pair)[i].empty()) { EXPECT_EQ(st.ok(), false); std::cout << "deserialize failed: " << st.to_json() << std::endl; @@ -256,11 +256,11 @@ TEST(FromStringTest, ScalaWrapperFieldVsDataType) { ReadBuffer rand_rb(rand_date.data(), rand_date.size()); auto col = data_type_ptr->create_column(); - Status st = data_type_ptr->from_string(min_rb, col); + Status st = data_type_ptr->from_string(min_rb, col.get()); EXPECT_EQ(st.ok(), true); - st = data_type_ptr->from_string(max_rb, col); + st = data_type_ptr->from_string(max_rb, col.get()); EXPECT_EQ(st.ok(), true); - st = data_type_ptr->from_string(rand_rb, col); + st = data_type_ptr->from_string(rand_rb, col.get()); EXPECT_EQ(st.ok(), true); string min_s_d = data_type_ptr->to_string(*col, 0); @@ -319,7 +319,7 @@ TEST(FromStringTest, ScalaWrapperFieldVsDataType) { string rand_ip = rand_wf->to_string(); ReadBuffer rand_rb(rand_ip.data(), rand_ip.size()); auto col = data_type_ptr->create_column(); - st = data_type_ptr->from_string(rand_rb, col); + st = data_type_ptr->from_string(rand_rb, col.get()); EXPECT_EQ(st.ok(), true); string rand_s_d = data_type_ptr->to_string(*col, 0); rtrim(rand_ip); @@ -336,7 +336,7 @@ TEST(FromStringTest, ScalaWrapperFieldVsDataType) { EXPECT_EQ(st.ok(), false); ReadBuffer rand_rb(pair.second.data(), pair.second.size()); auto col = data_type_ptr->create_column(); - st = data_type_ptr->from_string(rand_rb, col); + st = data_type_ptr->from_string(rand_rb, col.get()); EXPECT_EQ(st.ok(), false); } } diff --git a/be/test/vec/data_types/serde/data_type_serde_text_test.cpp b/be/test/vec/data_types/serde/data_type_serde_text_test.cpp index 2affbc36c86..b65b3fc6f63 100644 --- a/be/test/vec/data_types/serde/data_type_serde_text_test.cpp +++ b/be/test/vec/data_types/serde/data_type_serde_text_test.cpp @@ -510,7 +510,7 @@ TEST(TextSerde, ComplexTypeSerdeTextTest) { { // from_string ReadBuffer rb(rand_str.data(), rand_str.size()); - Status status = array_data_type_ptr->from_string(rb, col2); + Status status = array_data_type_ptr->from_string(rb, col2.get()); EXPECT_EQ(status.ok(), true); auto ser_col = ColumnString::create(); ser_col->reserve(1); @@ -661,7 +661,7 @@ TEST(TextSerde, ComplexTypeSerdeTextTest) { { ReadBuffer rb(rand_str.data(), rand_str.size()); std::cout << "from string rb: " << rb.to_string() << std::endl; - Status stat = map_data_type_ptr->from_string(rb, col2); + Status stat = map_data_type_ptr->from_string(rb, col2.get()); std::cout << stat.to_json() << std::endl; auto ser_col = ColumnString::create(); ser_col->reserve(1); @@ -840,7 +840,7 @@ TEST(TextSerde, ComplexTypeWithNestedSerdeTextTest) { // from_string ReadBuffer rb(rand_str.data(), rand_str.size()); auto col2 = array_data_type_ptr->create_column(); - Status status = array_data_type_ptr->from_string(rb, col2); + Status status = array_data_type_ptr->from_string(rb, col2.get()); if (expect_from_string_str == "") { EXPECT_EQ(status.ok(), false); std::cout << "test from_string: " << status.to_json() << std::endl; @@ -995,7 +995,7 @@ TEST(TextSerde, ComplexTypeWithNestedSerdeTextTest) { // from_string ReadBuffer rb(rand_str.data(), rand_str.size()); auto col2 = array_data_type_ptr->create_column(); - Status status = array_data_type_ptr->from_string(rb, col2); + Status status = array_data_type_ptr->from_string(rb, col2.get()); if (expect_from_string_str == "") { EXPECT_EQ(status.ok(), false); std::cout << "test from_string: " << status.to_json() << std::endl; @@ -1213,7 +1213,7 @@ TEST(TextSerde, ComplexTypeWithNestedSerdeTextTest) { // from_string ReadBuffer rb(rand_str.data(), rand_str.size()); auto col2 = map_data_type_ptr->create_column(); - Status status = map_data_type_ptr->from_string(rb, col2); + Status status = map_data_type_ptr->from_string(rb, col2.get()); if (expect_from_string_str == "") { EXPECT_EQ(status.ok(), false); std::cout << "test from_string: " << status.to_json() << std::endl; @@ -1354,7 +1354,7 @@ TEST(TextSerde, ComplexTypeWithNestedSerdeTextTest) { // from_string ReadBuffer rb(rand_str.data(), rand_str.size()); auto col2 = array_data_type_ptr->create_column(); - Status status = array_data_type_ptr->from_string(rb, col2); + Status status = array_data_type_ptr->from_string(rb, col2.get()); if (expect_from_string_str == "") { EXPECT_EQ(status.ok(), false); std::cout << "test from_string: " << status.to_json() << std::endl; diff --git a/be/test/vec/function/function_test_util.h b/be/test/vec/function/function_test_util.h index c33a1d64f83..a3809bf8ec6 100644 --- a/be/test/vec/function/function_test_util.h +++ b/be/test/vec/function/function_test_util.h @@ -315,7 +315,7 @@ Status check_function(const std::string& func_name, const InputTypeSet& input_ty // 3. check the result of function ColumnPtr column = block.get_columns()[result]; - EXPECT_TRUE(column != nullptr); + EXPECT_TRUE(column); for (int i = 0; i < row_size; ++i) { // update current line diff --git a/be/test/vec/olap/char_type_padding_test.cpp b/be/test/vec/olap/char_type_padding_test.cpp index 0e4879e46a6..dfdfea3026e 100644 --- a/be/test/vec/olap/char_type_padding_test.cpp +++ b/be/test/vec/olap/char_type_padding_test.cpp @@ -40,10 +40,10 @@ TEST(CharTypePaddingTest, CharTypePaddingFullTest) { for (size_t i = 0; i < rows; i++) { input->insert_data(str.data(), str.length()); } - EXPECT_FALSE(ConvertorChar::should_padding(input, str.length())); + EXPECT_FALSE(ConvertorChar::should_padding(input.get(), str.length())); input->insert_data(str.data(), str.length() - 1); - EXPECT_TRUE(ConvertorChar::should_padding(input, str.length())); + EXPECT_TRUE(ConvertorChar::should_padding(input.get(), str.length())); } TEST(CharTypePaddingTest, CharTypePaddingDataTest) { @@ -56,7 +56,7 @@ TEST(CharTypePaddingTest, CharTypePaddingDataTest) { input->insert_data(str.data(), str.length() - i); } - auto output = ConvertorChar::clone_and_padding(input, str.length()); + auto output = ConvertorChar::clone_and_padding(input.get(), str.length()); for (int i = 0; i < rows; i++) { auto cell = output->get_data_at(i).to_string(); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org