From 517aa90f29a930dfa23dbe1229e08b9a8f749ea8 Mon Sep 17 00:00:00 2001 From: zhaochangle Date: Fri, 19 Apr 2024 00:02:25 +0800 Subject: [PATCH] 1 --- be/src/exec/rowid_fetcher.h | 8 ++++--- be/src/exprs/hybrid_set.h | 4 ++++ be/src/olap/comparison_predicate.h | 3 +++ be/src/olap/in_list_predicate.h | 2 ++ be/src/olap/like_column_predicate.h | 2 ++ be/src/olap/match_predicate.h | 1 + be/src/olap/page_cache.h | 1 + .../segment_v2/hierarchical_data_reader.cpp | 2 ++ be/src/pipeline/exec/exchange_sink_buffer.cpp | 1 + be/src/pipeline/exec/exchange_sink_buffer.h | 1 + .../pipeline/exec/set_probe_sink_operator.cpp | 2 ++ be/src/pipeline/exec/set_sink_operator.cpp | 1 + be/src/pipeline/exec/set_source_operator.cpp | 1 + be/src/runtime/jsonb_value.h | 11 +++++++++ be/src/runtime/primitive_type.h | 10 +++++--- be/src/runtime/snapshot_loader.cpp | 1 + be/src/util/easy_json.cc | 1 + be/src/util/timezone_utils.cpp | 1 + .../aggregate_function_orthogonal_bitmap.h | 16 +++++-------- .../aggregate_function_percentile_approx.h | 1 + .../aggregate_function_reader_first_last.h | 4 ++++ .../aggregate_function_window.h | 4 ++++ be/src/vec/columns/column.h | 4 ++++ be/src/vec/columns/column_array.cpp | 4 ++++ be/src/vec/columns/column_complex.h | 3 +++ be/src/vec/columns/column_const.h | 1 + be/src/vec/columns/column_decimal.cpp | 1 + be/src/vec/columns/column_dictionary.h | 12 ++++++++++ be/src/vec/columns/column_dummy.h | 5 ++++ .../vec/columns/column_fixed_length_object.h | 6 +++++ be/src/vec/columns/column_map.cpp | 1 + be/src/vec/columns/column_map.h | 1 + be/src/vec/columns/column_nullable.cpp | 2 ++ be/src/vec/columns/column_nullable.h | 1 + be/src/vec/columns/column_object.h | 3 +++ be/src/vec/columns/column_string.cpp | 1 + be/src/vec/columns/column_string.h | 16 +++++-------- be/src/vec/columns/column_struct.cpp | 4 ++++ be/src/vec/columns/column_struct.h | 1 + be/src/vec/columns/column_vector.cpp | 1 + be/src/vec/columns/column_vector.h | 1 + be/src/vec/columns/columns_common.h | 1 + be/src/vec/columns/predicate_column.h | 15 ++++++++++++ be/src/vec/common/field_visitors.h | 1 + be/src/vec/core/decimal_comparison.h | 6 +++++ be/src/vec/core/field.h | 11 +++++++++ be/src/vec/data_types/data_type_decimal.h | 1 + be/src/vec/data_types/data_type_nothing.cpp | 2 ++ be/src/vec/data_types/data_type_struct.cpp | 3 +++ be/src/vec/data_types/data_type_struct.h | 1 + .../data_types/serde/data_type_number_serde.h | 1 + .../format/parquet/byte_array_dict_decoder.h | 16 ++++--------- be/src/vec/exec/format/parquet/decoder.h | 24 ++++++++++--------- .../format/parquet/parquet_column_convert.h | 2 +- .../parquet/vparquet_column_chunk_reader.h | 13 ++++++---- .../format/parquet/vparquet_column_reader.cpp | 9 ------- .../format/parquet/vparquet_column_reader.h | 17 ++++++------- be/src/vec/exec/format/table/iceberg_reader.h | 11 +++++---- be/src/vec/exec/vdata_gen_scan_node.cpp | 1 + be/src/vec/exec/vset_operation_node.cpp | 4 ++++ .../vec/exprs/table_function/vexplode_split.h | 17 ++++++------- .../array/function_array_enumerate_uniq.cpp | 1 + .../functions/array/function_arrays_overlap.h | 10 ++++---- be/src/vec/functions/function.h | 1 + be/src/vec/functions/function_cast.h | 2 ++ be/src/vec/functions/function_jsonb.cpp | 1 + .../functions/function_variadic_arguments.h | 1 + be/src/vec/functions/in.h | 23 ++++++++++-------- be/src/vec/json/parse2column.h | 13 +++++----- be/src/vec/jsonb/serialize.h | 2 +- be/src/vec/olap/olap_data_convertor.h | 2 ++ be/src/vec/runtime/vdatetime_value.cpp | 2 ++ 72 files changed, 251 insertions(+), 109 deletions(-) diff --git a/be/src/exec/rowid_fetcher.h b/be/src/exec/rowid_fetcher.h index 7ca2ef19143b0a..1fc8b02a6796cb 100644 --- a/be/src/exec/rowid_fetcher.h +++ b/be/src/exec/rowid_fetcher.h @@ -22,12 +22,11 @@ #include #include +#include #include #include "common/status.h" #include "exec/tablet_info.h" // DorisNodesInfo -#include "olap/storage_engine.h" -#include "vec/columns/column_string.h" #include "vec/core/block.h" #include "vec/data_types/data_type.h" @@ -38,6 +37,9 @@ class RuntimeState; class TupleDescriptor; namespace vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; class MutableBlock; } // namespace vectorized @@ -52,7 +54,7 @@ struct FetchOption { class RowIDFetcher { public: - RowIDFetcher(const FetchOption& fetch_opt) : _fetch_option(fetch_opt) {} + RowIDFetcher(FetchOption fetch_opt) : _fetch_option(std::move(fetch_opt)) {} Status init(); Status fetch(const vectorized::ColumnPtr& row_ids, vectorized::Block* block); diff --git a/be/src/exprs/hybrid_set.h b/be/src/exprs/hybrid_set.h index 03cde80c24bf13..26f6d63deeee17 100644 --- a/be/src/exprs/hybrid_set.h +++ b/be/src/exprs/hybrid_set.h @@ -203,23 +203,27 @@ class HybridSetBase : public RuntimeFilterFuncBase { virtual void find_batch(const doris::vectorized::IColumn& column, size_t rows, doris::vectorized::ColumnUInt8::Container& results) { LOG(FATAL) << "HybridSetBase not support find_batch"; + __builtin_unreachable(); } virtual void find_batch_negative(const doris::vectorized::IColumn& column, size_t rows, doris::vectorized::ColumnUInt8::Container& results) { LOG(FATAL) << "HybridSetBase not support find_batch_negative"; + __builtin_unreachable(); } virtual void find_batch_nullable(const doris::vectorized::IColumn& column, size_t rows, const doris::vectorized::NullMap& null_map, doris::vectorized::ColumnUInt8::Container& results) { LOG(FATAL) << "HybridSetBase not support find_batch_nullable"; + __builtin_unreachable(); } virtual void find_batch_nullable_negative(const doris::vectorized::IColumn& column, size_t rows, const doris::vectorized::NullMap& null_map, doris::vectorized::ColumnUInt8::Container& results) { LOG(FATAL) << "HybridSetBase not support find_batch_nullable_negative"; + __builtin_unreachable(); } class IteratorBase { diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h index 19e92a41bca5f7..6a5f27bd32606c 100644 --- a/be/src/olap/comparison_predicate.h +++ b/be/src/olap/comparison_predicate.h @@ -283,6 +283,7 @@ class ComparisonPredicateBase : public ColumnPredicate { } while (false); } else { LOG(FATAL) << "column_dictionary must use StringRef predicate."; + __builtin_unreachable(); } } else { auto* data_array = @@ -313,6 +314,7 @@ class ComparisonPredicateBase : public ColumnPredicate { } while (false); } else { LOG(FATAL) << "column_dictionary must use StringRef predicate."; + __builtin_unreachable(); } } else { auto* data_array = @@ -518,6 +520,7 @@ class ComparisonPredicateBase : public ColumnPredicate { dict_code); } else { LOG(FATAL) << "column_dictionary must use StringRef predicate."; + __builtin_unreachable(); } } else { auto* data_array = diff --git a/be/src/olap/in_list_predicate.h b/be/src/olap/in_list_predicate.h index ec0e770ebd7b62..4a1a10f898f0e6 100644 --- a/be/src/olap/in_list_predicate.h +++ b/be/src/olap/in_list_predicate.h @@ -427,6 +427,7 @@ class InListPredicateBase : public ColumnPredicate { } } else { LOG(FATAL) << "column_dictionary must use StringRef predicate."; + __builtin_unreachable(); } } else { auto& pred_col = @@ -490,6 +491,7 @@ class InListPredicateBase : public ColumnPredicate { } } else { LOG(FATAL) << "column_dictionary must use StringRef predicate."; + __builtin_unreachable(); } } else { auto* nested_col_ptr = vectorized::check_and_get_column< diff --git a/be/src/olap/like_column_predicate.h b/be/src/olap/like_column_predicate.h index eda033095a0e3a..31763d45f7edc7 100644 --- a/be/src/olap/like_column_predicate.h +++ b/be/src/olap/like_column_predicate.h @@ -129,6 +129,7 @@ class LikeColumnPredicate : public ColumnPredicate { } } else { LOG(FATAL) << "vectorized (not) like predicates should be dict column"; + __builtin_unreachable(); } } else { if (column.is_column_dictionary()) { @@ -153,6 +154,7 @@ class LikeColumnPredicate : public ColumnPredicate { } } else { LOG(FATAL) << "vectorized (not) like predicates should be dict column"; + __builtin_unreachable(); } } } diff --git a/be/src/olap/match_predicate.h b/be/src/olap/match_predicate.h index 5ec8a8bf9cbf97..862bc4a0f59a63 100644 --- a/be/src/olap/match_predicate.h +++ b/be/src/olap/match_predicate.h @@ -56,6 +56,7 @@ class MatchPredicate : public ColumnPredicate { Status evaluate(BitmapIndexIterator* iterator, uint32_t num_rows, roaring::Roaring* roaring) const override { LOG(FATAL) << "Not Implemented MatchPredicate::evaluate"; + __builtin_unreachable(); } //evaluate predicate on inverted diff --git a/be/src/olap/page_cache.h b/be/src/olap/page_cache.h index 9b36bfc690f61a..9799b06b1a6be4 100644 --- a/be/src/olap/page_cache.h +++ b/be/src/olap/page_cache.h @@ -196,6 +196,7 @@ class StoragePageCache { } default: LOG(FATAL) << "get error type page cache"; + __builtin_unreachable(); } LOG(FATAL) << "__builtin_unreachable"; __builtin_unreachable(); diff --git a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp index 2feb865de28418..66ad0eb92a996d 100644 --- a/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp +++ b/be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp @@ -76,6 +76,7 @@ Status HierarchicalDataReader::init(const ColumnIteratorOptions& opts) { Status HierarchicalDataReader::seek_to_first() { LOG(FATAL) << "Not implemented"; + __builtin_unreachable(); } Status HierarchicalDataReader::seek_to_ordinal(ordinal_t ord) { @@ -154,6 +155,7 @@ Status ExtractReader::init(const ColumnIteratorOptions& opts) { Status ExtractReader::seek_to_first() { LOG(FATAL) << "Not implemented"; + __builtin_unreachable(); } Status ExtractReader::seek_to_ordinal(ordinal_t ord) { diff --git a/be/src/pipeline/exec/exchange_sink_buffer.cpp b/be/src/pipeline/exec/exchange_sink_buffer.cpp index 44b655150af8f2..7f46bfcf353341 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.cpp +++ b/be/src/pipeline/exec/exchange_sink_buffer.cpp @@ -440,6 +440,7 @@ void ExchangeSinkBuffer::_ended(InstanceLoId id) { LOG(INFO) << ss.str(); LOG(FATAL) << "not find the instance id"; + __builtin_unreachable(); } else { std::unique_lock lock(*_instance_to_package_queue_mutex[id]); if (!_rpc_channel_is_idle[id]) { diff --git a/be/src/pipeline/exec/exchange_sink_buffer.h b/be/src/pipeline/exec/exchange_sink_buffer.h index 43fdc98d24c00e..74207bbffd3305 100644 --- a/be/src/pipeline/exec/exchange_sink_buffer.h +++ b/be/src/pipeline/exec/exchange_sink_buffer.h @@ -177,6 +177,7 @@ class ExchangeSendCallback : public ::doris::DummyBrpcCallback { LOG(FATAL) << "brpc callback error: " << exp.what(); } catch (...) { LOG(FATAL) << "brpc callback error."; + __builtin_unreachable(); } } int64_t start_rpc_time; diff --git a/be/src/pipeline/exec/set_probe_sink_operator.cpp b/be/src/pipeline/exec/set_probe_sink_operator.cpp index 9ae40930d5f40a..744c8b17e2257d 100644 --- a/be/src/pipeline/exec/set_probe_sink_operator.cpp +++ b/be/src/pipeline/exec/set_probe_sink_operator.cpp @@ -124,6 +124,7 @@ Status SetProbeSinkOperatorX::sink(RuntimeState* state, vectorized return process_hashtable_ctx.mark_data_in_hashtable(arg); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *local_state._shared_state->hash_table_variants)); @@ -283,6 +284,7 @@ void SetProbeSinkOperatorX::_refresh_hash_table( } } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *hash_table_variants); diff --git a/be/src/pipeline/exec/set_sink_operator.cpp b/be/src/pipeline/exec/set_sink_operator.cpp index 3b6de314060efb..2042e3eb1a1d51 100644 --- a/be/src/pipeline/exec/set_sink_operator.cpp +++ b/be/src/pipeline/exec/set_sink_operator.cpp @@ -122,6 +122,7 @@ Status SetSinkOperatorX::_process_build_block( static_cast(hash_table_build_process(arg, local_state._arena)); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *local_state._shared_state->hash_table_variants); diff --git a/be/src/pipeline/exec/set_source_operator.cpp b/be/src/pipeline/exec/set_source_operator.cpp index 97ad66a867eebd..88d38d325af003 100644 --- a/be/src/pipeline/exec/set_source_operator.cpp +++ b/be/src/pipeline/exec/set_source_operator.cpp @@ -102,6 +102,7 @@ Status SetSourceOperatorX::get_block(RuntimeState* state, vectoriz state->batch_size(), eos); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *local_state._shared_state->hash_table_variants); diff --git a/be/src/runtime/jsonb_value.h b/be/src/runtime/jsonb_value.h index 1c521999388a91..1df9469e1720cd 100644 --- a/be/src/runtime/jsonb_value.h +++ b/be/src/runtime/jsonb_value.h @@ -62,46 +62,57 @@ struct JsonBinaryValue { bool operator==(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } // != bool ne(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } // <= bool le(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } // >= bool ge(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } // < bool lt(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } // > bool gt(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } bool operator!=(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } bool operator<=(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } bool operator>=(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } bool operator<(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } bool operator>(const JsonBinaryValue& other) const { LOG(FATAL) << "comparing between JsonBinaryValue is not supported"; + __builtin_unreachable(); } Status from_json_string(const char* s, int len); diff --git a/be/src/runtime/primitive_type.h b/be/src/runtime/primitive_type.h index 0e74a86d76c94e..59c0d91e432391 100644 --- a/be/src/runtime/primitive_type.h +++ b/be/src/runtime/primitive_type.h @@ -20,15 +20,13 @@ #include #include #include -#include +#include #include -#include #include "olap/decimal12.h" #include "runtime/define_primitive_type.h" #include "vec/columns/column_decimal.h" -#include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/core/types.h" @@ -37,6 +35,12 @@ namespace doris { +namespace vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; +} // namespace vectorized + class DecimalV2Value; struct StringRef; struct JsonBinaryValue; diff --git a/be/src/runtime/snapshot_loader.cpp b/be/src/runtime/snapshot_loader.cpp index 36c587934dbcb9..acc2015132e688 100644 --- a/be/src/runtime/snapshot_loader.cpp +++ b/be/src/runtime/snapshot_loader.cpp @@ -771,6 +771,7 @@ Status SnapshotLoader::move(const std::string& snapshot_path, TabletSharedPtr ta } else { LOG(FATAL) << "only support overwrite now"; + __builtin_unreachable(); } // snapshot loader not need to change tablet uid diff --git a/be/src/util/easy_json.cc b/be/src/util/easy_json.cc index 4df9e88fcee056..46c3a1867f7b42 100644 --- a/be/src/util/easy_json.cc +++ b/be/src/util/easy_json.cc @@ -201,6 +201,7 @@ EasyJson EasyJson::PushBack(EasyJson::ComplexTypeInitializer val) { push_val.SetArray(); } else { LOG(FATAL) << "Unknown initializer type"; + __builtin_unreachable(); } value_->PushBack(push_val, alloc_->allocator()); return EasyJson(&(*value_)[value_->Size() - 1], alloc_); diff --git a/be/src/util/timezone_utils.cpp b/be/src/util/timezone_utils.cpp index 16827a6504ca17..6d561f2151ecf1 100644 --- a/be/src/util/timezone_utils.cpp +++ b/be/src/util/timezone_utils.cpp @@ -120,6 +120,7 @@ T next_from_charstream(int8_t*& src) { return value; } else { LOG(FATAL) << "Unknown endianess"; + __builtin_unreachable(); } LOG(FATAL) << "__builtin_unreachable"; __builtin_unreachable(); diff --git a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h index 5877020eaf92eb..d7de66681ed110 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h +++ b/be/src/vec/aggregate_functions/aggregate_function_orthogonal_bitmap.h @@ -18,11 +18,10 @@ #pragma once #include -#include -#include -#include #include +#include +#include #include #include #include @@ -33,23 +32,20 @@ #include "util/bitmap_value.h" #include "vec/aggregate_functions/aggregate_function.h" #include "vec/columns/column_complex.h" -#include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/core/types.h" #include "vec/data_types/data_type_bitmap.h" #include "vec/data_types/data_type_number.h" #include "vec/io/io_helper.h" -namespace doris { -namespace vectorized { +namespace doris::vectorized { class Arena; class BufferReadable; class BufferWritable; class IColumn; -} // namespace vectorized -} // namespace doris - -namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; template struct AggOrthBitmapBaseData { diff --git a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h index 2eb7cc33098c32..b89c273dd77af2 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h +++ b/be/src/vec/aggregate_functions/aggregate_function_percentile_approx.h @@ -203,6 +203,7 @@ class AggregateFunctionPercentileApproxMerge : public AggregateFunctionPercentil void add(AggregateDataPtr __restrict place, const IColumn** columns, ssize_t row_num, Arena*) const override { LOG(FATAL) << "AggregateFunctionPercentileApproxMerge do not support add()"; + __builtin_unreachable(); } }; diff --git a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h index bbf62b09222857..cbaaa2849332f4 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h +++ b/be/src/vec/aggregate_functions/aggregate_function_reader_first_last.h @@ -226,15 +226,19 @@ class ReaderFunctionData final int64_t frame_end, AggregateDataPtr place, const IColumn** columns, Arena* arena) const override { LOG(FATAL) << "ReaderFunctionData do not support add_range_single_place"; + __builtin_unreachable(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena*) const override { LOG(FATAL) << "ReaderFunctionData do not support merge"; + __builtin_unreachable(); } void serialize(ConstAggregateDataPtr place, BufferWritable& buf) const override { LOG(FATAL) << "ReaderFunctionData do not support serialize"; + __builtin_unreachable(); } void deserialize(AggregateDataPtr place, BufferReadable& buf, Arena*) const override { LOG(FATAL) << "ReaderFunctionData do not support deserialize"; + __builtin_unreachable(); } private: diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h index 3b0748d519f0af..e7b8a70ed0b78a 100644 --- a/be/src/vec/aggregate_functions/aggregate_function_window.h +++ b/be/src/vec/aggregate_functions/aggregate_function_window.h @@ -559,15 +559,19 @@ class WindowFunctionData final void add(AggregateDataPtr place, const IColumn** columns, ssize_t row_num, Arena* arena) const override { LOG(FATAL) << "WindowFunctionLeadLagData do not support add"; + __builtin_unreachable(); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena*) const override { LOG(FATAL) << "WindowFunctionLeadLagData do not support merge"; + __builtin_unreachable(); } void serialize(ConstAggregateDataPtr place, BufferWritable& buf) const override { LOG(FATAL) << "WindowFunctionLeadLagData do not support serialize"; + __builtin_unreachable(); } void deserialize(AggregateDataPtr place, BufferReadable& buf, Arena*) const override { LOG(FATAL) << "WindowFunctionLeadLagData do not support deserialize"; + __builtin_unreachable(); } private: diff --git a/be/src/vec/columns/column.h b/be/src/vec/columns/column.h index 8fdc01701c8e90..950eb53cde4485 100644 --- a/be/src/vec/columns/column.h +++ b/be/src/vec/columns/column.h @@ -339,22 +339,26 @@ class IColumn : public COW { virtual void serialize_vec(std::vector& keys, size_t num_rows, size_t max_row_byte_size) const { LOG(FATAL) << "serialize_vec not supported"; + __builtin_unreachable(); } virtual void serialize_vec_with_null_map(std::vector& keys, size_t num_rows, const uint8_t* null_map) const { LOG(FATAL) << "serialize_vec_with_null_map not supported"; + __builtin_unreachable(); } // This function deserializes group-by keys into column in the vectorized way. virtual void deserialize_vec(std::vector& keys, const size_t num_rows) { LOG(FATAL) << "deserialize_vec not supported"; + __builtin_unreachable(); } // Used in ColumnNullable::deserialize_vec virtual void deserialize_vec_with_null_map(std::vector& keys, const size_t num_rows, const uint8_t* null_map) { LOG(FATAL) << "deserialize_vec_with_null_map not supported"; + __builtin_unreachable(); } /// TODO: SipHash is slower than city or xx hash, rethink we should have a new interface diff --git a/be/src/vec/columns/column_array.cpp b/be/src/vec/columns/column_array.cpp index f8eb146bdb5bf5..591eb74ca09178 100644 --- a/be/src/vec/columns/column_array.cpp +++ b/be/src/vec/columns/column_array.cpp @@ -93,6 +93,7 @@ ColumnArray::ColumnArray(MutableColumnPtr&& nested_column, MutableColumnPtr&& of if (!offsets_concrete) { LOG(FATAL) << "offsets_column must be a ColumnUInt64"; + __builtin_unreachable(); } if (!offsets_concrete->empty() && data) { @@ -114,6 +115,7 @@ ColumnArray::ColumnArray(MutableColumnPtr&& nested_column, MutableColumnPtr&& of ColumnArray::ColumnArray(MutableColumnPtr&& nested_column) : data(std::move(nested_column)) { if (!data->empty()) { LOG(FATAL) << "Not empty data passed to ColumnArray, but no offsets passed"; + __builtin_unreachable(); } offsets = ColumnOffsets::create(); @@ -239,6 +241,7 @@ void ColumnArray::insert_data(const char* pos, size_t length) { if (pos != end) LOG(FATAL) << "Incorrect length argument for method ColumnArray::insert_data"; + __builtin_unreachable(); } get_offsets().push_back(get_offsets().back() + elems); @@ -1101,6 +1104,7 @@ ColumnPtr ColumnArray::permute(const Permutation& perm, size_t limit) const { } if (perm.size() < limit) { LOG(FATAL) << "Size of permutation is less than required."; + __builtin_unreachable(); } if (limit == 0) { return ColumnArray::create(data); diff --git a/be/src/vec/columns/column_complex.h b/be/src/vec/columns/column_complex.h index 54b0650e8003de..193834c80f1467 100644 --- a/be/src/vec/columns/column_complex.h +++ b/be/src/vec/columns/column_complex.h @@ -82,6 +82,7 @@ class ColumnComplexType final : public COWHelper> pvalue->deserialize(Slice(pos, length)); } else { LOG(FATAL) << "Unexpected type in column complex"; + __builtin_unreachable(); } } @@ -129,6 +130,7 @@ class ColumnComplexType final : public COWHelper> void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, size_t limit) const override { LOG(FATAL) << "get_indices_of_non_default_rows not implemented"; + __builtin_unreachable(); } [[noreturn]] ColumnPtr index(const IColumn& indexes, size_t limit) const override { LOG(FATAL) << "index not implemented"; @@ -358,6 +360,7 @@ ColumnPtr ColumnComplexType::permute(const IColumn::Permutation& perm, size_t if (perm.size() < limit) { LOG(FATAL) << "Size of permutation is less than required."; + __builtin_unreachable(); } auto res = this->create(limit); diff --git a/be/src/vec/columns/column_const.h b/be/src/vec/columns/column_const.h index e1c3c52949bd04..d5e3223acfcdbb 100644 --- a/be/src/vec/columns/column_const.h +++ b/be/src/vec/columns/column_const.h @@ -296,6 +296,7 @@ class ColumnConst final : public COWHelper { void replace_column_data_default(size_t self_row = 0) override { DCHECK(size() > self_row); LOG(FATAL) << "should not call the method in column const"; + __builtin_unreachable(); } }; } // namespace doris::vectorized diff --git a/be/src/vec/columns/column_decimal.cpp b/be/src/vec/columns/column_decimal.cpp index 9a05a8d68dfd3c..5d90b66b8e261a 100644 --- a/be/src/vec/columns/column_decimal.cpp +++ b/be/src/vec/columns/column_decimal.cpp @@ -236,6 +236,7 @@ ColumnPtr ColumnDecimal::permute(const IColumn::Permutation& perm, size_t lim size_t size = limit ? std::min(data.size(), limit) : data.size(); if (perm.size() < size) { LOG(FATAL) << "Size of permutation is less than required."; + __builtin_unreachable(); } auto res = this->create(size, scale); diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h index 3b1537b83d3a8a..7f10e71f858eec 100644 --- a/be/src/vec/columns/column_dictionary.h +++ b/be/src/vec/columns/column_dictionary.h @@ -72,25 +72,30 @@ class ColumnDictionary final : public COWHelper> { void insert_from(const IColumn& src, size_t n) override { LOG(FATAL) << "insert_from not supported in ColumnDictionary"; + __builtin_unreachable(); } void insert_range_from(const IColumn& src, size_t start, size_t length) override { LOG(FATAL) << "insert_range_from not supported in ColumnDictionary"; + __builtin_unreachable(); } void insert_indices_from(const IColumn& src, const uint32_t* indices_begin, const uint32_t* indices_end) override { LOG(FATAL) << "insert_indices_from not supported in ColumnDictionary"; + __builtin_unreachable(); } void pop_back(size_t n) override { LOG(FATAL) << "pop_back not supported in ColumnDictionary"; } void update_hash_with_value(size_t n, SipHash& hash) const override { LOG(FATAL) << "update_hash_with_value not supported in ColumnDictionary"; + __builtin_unreachable(); } void insert_data(const char* pos, size_t /*length*/) override { LOG(FATAL) << "insert_data not supported in ColumnDictionary"; + __builtin_unreachable(); } void insert_default() override { _codes.push_back(_dict.get_null_code()); } @@ -109,6 +114,7 @@ class ColumnDictionary final : public COWHelper> { void get_permutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation& res) const override { LOG(FATAL) << "get_permutation not supported in ColumnDictionary"; + __builtin_unreachable(); } void reserve(size_t n) override { _codes.reserve(n); } @@ -122,6 +128,7 @@ class ColumnDictionary final : public COWHelper> { void insert(const Field& x) override { LOG(FATAL) << "insert not supported in ColumnDictionary"; + __builtin_unreachable(); } Field operator[](size_t n) const override { return _codes[n]; } @@ -155,6 +162,7 @@ class ColumnDictionary final : public COWHelper> { void get_indices_of_non_default_rows(IColumn::Offsets64& indices, size_t from, size_t limit) const override { LOG(FATAL) << "get_indices_of_non_default_rows not supported in ColumnDictionary"; + __builtin_unreachable(); } size_t size_of_value_if_fixed() const override { return sizeof(T); } @@ -193,11 +201,13 @@ class ColumnDictionary final : public COWHelper> { void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector) const override { LOG(FATAL) << "append_data_by_selector is not supported in ColumnDictionary!"; + __builtin_unreachable(); } void append_data_by_selector(MutableColumnPtr& res, const IColumn::Selector& selector, size_t begin, size_t end) const override { LOG(FATAL) << "append_data_by_selector is not supported in ColumnDictionary!"; + __builtin_unreachable(); } [[noreturn]] ColumnPtr index(const IColumn& indexes, size_t limit) const override { @@ -223,10 +233,12 @@ class ColumnDictionary final : public COWHelper> { void replace_column_data(const IColumn&, size_t row, size_t self_row = 0) override { LOG(FATAL) << "should not call replace_column_data in ColumnDictionary"; + __builtin_unreachable(); } void replace_column_data_default(size_t self_row = 0) override { LOG(FATAL) << "should not call replace_column_data_default in ColumnDictionary"; + __builtin_unreachable(); } /** diff --git a/be/src/vec/columns/column_dummy.h b/be/src/vec/columns/column_dummy.h index a28133c69a3d80..403cd6228e177d 100644 --- a/be/src/vec/columns/column_dummy.h +++ b/be/src/vec/columns/column_dummy.h @@ -99,6 +99,7 @@ class IColumnDummy : public IColumn { ColumnPtr permute(const Permutation& perm, size_t limit) const override { if (s != perm.size()) { LOG(FATAL) << "Size of permutation doesn't match size of column."; + __builtin_unreachable(); } return clone_dummy(limit ? std::min(s, limit) : s); @@ -148,19 +149,23 @@ class IColumnDummy : public IColumn { void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { LOG(FATAL) << "should not call the method in column dummy"; + __builtin_unreachable(); } void replace_column_data_default(size_t self_row = 0) override { LOG(FATAL) << "should not call the method in column dummy"; + __builtin_unreachable(); } void get_indices_of_non_default_rows(Offsets64&, size_t, size_t) const override { LOG(FATAL) << "should not call the method in column dummy"; + __builtin_unreachable(); } ColumnPtr index(const IColumn& indexes, size_t limit) const override { if (indexes.size() < limit) { LOG(FATAL) << "Size of indexes is less than required."; + __builtin_unreachable(); } return clone_dummy(limit ? limit : s); } diff --git a/be/src/vec/columns/column_fixed_length_object.h b/be/src/vec/columns/column_fixed_length_object.h index a10204ed48b0e3..8bb9062b2f513a 100644 --- a/be/src/vec/columns/column_fixed_length_object.h +++ b/be/src/vec/columns/column_fixed_length_object.h @@ -164,10 +164,12 @@ class ColumnFixedLengthObject final : public COWHelperempty() && keys_column && values_column) { diff --git a/be/src/vec/columns/column_map.h b/be/src/vec/columns/column_map.h index 73c13edd90a133..d0cd9e588a2192 100644 --- a/be/src/vec/columns/column_map.h +++ b/be/src/vec/columns/column_map.h @@ -126,6 +126,7 @@ class ColumnMap final : public COWHelper { void get_permutation(bool reverse, size_t limit, int nan_direction_hint, Permutation& res) const override { LOG(FATAL) << "get_permutation not implemented"; + __builtin_unreachable(); } void insert_indices_from(const IColumn& src, const uint32_t* indices_begin, diff --git a/be/src/vec/columns/column_nullable.cpp b/be/src/vec/columns/column_nullable.cpp index 0a4321eea53c0a..e80e9d9b636d52 100644 --- a/be/src/vec/columns/column_nullable.cpp +++ b/be/src/vec/columns/column_nullable.cpp @@ -44,6 +44,7 @@ ColumnNullable::ColumnNullable(MutableColumnPtr&& nested_column_, MutableColumnP if (is_column_const(*null_map)) { LOG(FATAL) << "ColumnNullable cannot have constant null map"; + __builtin_unreachable(); } _need_update_has_null = true; } @@ -524,6 +525,7 @@ void ColumnNullable::apply_null_map_impl(const ColumnUInt8& map) { if (arr1.size() != arr2.size()) { LOG(FATAL) << "Inconsistent sizes of ColumnNullable objects"; + __builtin_unreachable(); } for (size_t i = 0, size = arr1.size(); i < size; ++i) { diff --git a/be/src/vec/columns/column_nullable.h b/be/src/vec/columns/column_nullable.h index 7662c23e036f94..8842cc67ef251c 100644 --- a/be/src/vec/columns/column_nullable.h +++ b/be/src/vec/columns/column_nullable.h @@ -356,6 +356,7 @@ class ColumnNullable final : public COWHelper { void replace_column_data_default(size_t self_row = 0) override { LOG(FATAL) << "should not call the method in column nullable"; + __builtin_unreachable(); } MutableColumnPtr convert_to_predicate_column_if_dictionary() override { diff --git a/be/src/vec/columns/column_object.h b/be/src/vec/columns/column_object.h index 02edde29cf55ca..8573428ff2bba2 100644 --- a/be/src/vec/columns/column_object.h +++ b/be/src/vec/columns/column_object.h @@ -443,6 +443,7 @@ class ColumnObject final : public COWHelper { void insert_data(const char* pos, size_t length) override { LOG(FATAL) << "should not call the method in column object"; + __builtin_unreachable(); } ColumnPtr filter(const Filter&, ssize_t) const override; @@ -461,6 +462,7 @@ class ColumnObject final : public COWHelper { void get_permutation(bool reverse, size_t limit, int nan_direction_hint, Permutation& res) const override { LOG(FATAL) << "should not call the method in column object"; + __builtin_unreachable(); } bool is_variable_length() const override { return true; } @@ -471,6 +473,7 @@ class ColumnObject final : public COWHelper { void get_indices_of_non_default_rows(Offsets64&, size_t, size_t) const override { LOG(FATAL) << "should not call the method in column object"; + __builtin_unreachable(); } template diff --git a/be/src/vec/columns/column_string.cpp b/be/src/vec/columns/column_string.cpp index 634c51c47c376d..843b2fa72a7eb4 100644 --- a/be/src/vec/columns/column_string.cpp +++ b/be/src/vec/columns/column_string.cpp @@ -296,6 +296,7 @@ ColumnPtr ColumnStr::permute(const IColumn::Permutation& perm, size_t limit) if (perm.size() < limit) { LOG(FATAL) << "Size of permutation is less than required."; + __builtin_unreachable(); } if (limit == 0) { diff --git a/be/src/vec/columns/column_string.h b/be/src/vec/columns/column_string.h index 19e62c1a59ae55..8aa531e466d423 100644 --- a/be/src/vec/columns/column_string.h +++ b/be/src/vec/columns/column_string.h @@ -21,10 +21,10 @@ #pragma once #include -#include #include #include +#include #include #include #include @@ -36,7 +36,6 @@ #include "runtime/define_primitive_type.h" #include "util/hash_util.hpp" #include "vec/columns/column.h" -#include "vec/columns/column_impl.h" #include "vec/common/assert_cast.h" #include "vec/common/cow.h" #include "vec/common/memcmp_small.h" @@ -47,14 +46,9 @@ #include "vec/core/field.h" #include "vec/core/types.h" -namespace doris { -namespace vectorized { +namespace doris::vectorized { class Arena; class ColumnSorter; -} // namespace vectorized -} // namespace doris - -namespace doris::vectorized { /** Column for String values. */ @@ -99,9 +93,9 @@ class ColumnStr final : public COWHelper> { template struct lessWithCollation; - ColumnStr() = default; + ColumnStr() = default; - ColumnStr(const ColumnStr& src) + ColumnStr(const ColumnStr& src) : offsets(src.offsets.begin(), src.offsets.end()), chars(src.chars.begin(), src.chars.end()) {} @@ -554,11 +548,13 @@ class ColumnStr final : public COWHelper> { void replace_column_data(const IColumn& rhs, size_t row, size_t self_row = 0) override { LOG(FATAL) << "Method replace_column_data is not supported for ColumnString"; + __builtin_unreachable(); } // should replace according to 0,1,2... ,size,0,1,2... void replace_column_data_default(size_t self_row = 0) override { LOG(FATAL) << "Method replace_column_data_default is not supported for ColumnString"; + __builtin_unreachable(); } void compare_internal(size_t rhs_row_id, const IColumn& rhs, int nan_direction_hint, diff --git a/be/src/vec/columns/column_struct.cpp b/be/src/vec/columns/column_struct.cpp index c0f2b3cbb7ad6d..7b25d9a912cb18 100644 --- a/be/src/vec/columns/column_struct.cpp +++ b/be/src/vec/columns/column_struct.cpp @@ -54,6 +54,7 @@ ColumnStruct::ColumnStruct(MutableColumns&& mutable_columns) { for (auto& column : mutable_columns) { if (is_column_const(*column)) { LOG(FATAL) << "ColumnStruct cannot have ColumnConst as its element"; + __builtin_unreachable(); } columns.push_back(std::move(column)); } @@ -63,6 +64,7 @@ ColumnStruct::Ptr ColumnStruct::create(const Columns& columns) { for (const auto& column : columns) { if (is_column_const(*column)) { LOG(FATAL) << "ColumnStruct cannot have ColumnConst as its element"; + __builtin_unreachable(); } } auto column_struct = ColumnStruct::create(MutableColumns()); @@ -74,6 +76,7 @@ ColumnStruct::Ptr ColumnStruct::create(const TupleColumns& tuple_columns) { for (const auto& column : tuple_columns) { if (is_column_const(*column)) { LOG(FATAL) << "ColumnStruct cannot have ColumnConst as its element"; + __builtin_unreachable(); } } auto column_struct = ColumnStruct::create(MutableColumns()); @@ -146,6 +149,7 @@ void ColumnStruct::insert_from(const IColumn& src_, size_t n) { const size_t tuple_size = columns.size(); if (src.columns.size() != tuple_size) { LOG(FATAL) << "Cannot insert value of different size into tuple."; + __builtin_unreachable(); } for (size_t i = 0; i < tuple_size; ++i) { diff --git a/be/src/vec/columns/column_struct.h b/be/src/vec/columns/column_struct.h index 51cf35ff406a11..b80c833edf7a7a 100644 --- a/be/src/vec/columns/column_struct.h +++ b/be/src/vec/columns/column_struct.h @@ -128,6 +128,7 @@ class ColumnStruct final : public COWHelper { void get_permutation(bool reverse, size_t limit, int nan_direction_hint, Permutation& res) const override { LOG(FATAL) << "get_permutation not implemented"; + __builtin_unreachable(); } void append_data_by_selector(MutableColumnPtr& res, const Selector& selector) const override { return append_data_by_selector_impl(res, selector); diff --git a/be/src/vec/columns/column_vector.cpp b/be/src/vec/columns/column_vector.cpp index 97fadcb4071052..5ad9fa7e6325d2 100644 --- a/be/src/vec/columns/column_vector.cpp +++ b/be/src/vec/columns/column_vector.cpp @@ -495,6 +495,7 @@ ColumnPtr ColumnVector::permute(const IColumn::Permutation& perm, size_t limi if (perm.size() < limit) { LOG(FATAL) << "Size of permutation is less than required."; + __builtin_unreachable(); } auto res = this->create(limit); diff --git a/be/src/vec/columns/column_vector.h b/be/src/vec/columns/column_vector.h index 0b7d8350c5f44f..ff1aed42071fe4 100644 --- a/be/src/vec/columns/column_vector.h +++ b/be/src/vec/columns/column_vector.h @@ -199,6 +199,7 @@ class ColumnVector final : public COWHelper> { } } else { LOG(FATAL) << "double column not support insert_range_of_integer"; + __builtin_unreachable(); } } diff --git a/be/src/vec/columns/columns_common.h b/be/src/vec/columns/columns_common.h index bd5985f9eca9c3..ac2b15743e8757 100644 --- a/be/src/vec/columns/columns_common.h +++ b/be/src/vec/columns/columns_common.h @@ -104,6 +104,7 @@ ColumnPtr select_index_impl(const Column& column, const IColumn& indexes, size_t if (indexes.size() < limit) { LOG(FATAL) << "Size of indexes is less than required."; + __builtin_unreachable(); } if (auto* data_uint8 = detail::get_indexes_data(indexes)) { diff --git a/be/src/vec/columns/predicate_column.h b/be/src/vec/columns/predicate_column.h index 0fdca54dc388f4..4480f6ee8cae5b 100644 --- a/be/src/vec/columns/predicate_column.h +++ b/be/src/vec/columns/predicate_column.h @@ -115,33 +115,40 @@ class PredicateColumnType final : public COWHelper::ResultType apply_visitor(Visitor&& visitor, F&& return visitor(field.template get()); default: LOG(FATAL) << "Bad type of Field"; + __builtin_unreachable(); return {}; } } diff --git a/be/src/vec/core/decimal_comparison.h b/be/src/vec/core/decimal_comparison.h index a24b85bc7fdce5..c5c1e40e2ff4df 100644 --- a/be/src/vec/core/decimal_comparison.h +++ b/be/src/vec/core/decimal_comparison.h @@ -107,6 +107,7 @@ class DecimalComparison { static const UInt32 max_scale = max_decimal_precision(); if (scale_a > max_scale || scale_b > max_scale) { LOG(FATAL) << "Bad scale of decimal field"; + __builtin_unreachable(); } Shift shift; @@ -213,6 +214,7 @@ class DecimalComparison { constant_vector(a, c1_vec->get_data(), vec_res, scale); else { LOG(FATAL) << "Wrong column in Decimal comparison"; + __builtin_unreachable(); } } else if (c1_is_const) { const ColumnConst* c1_const = check_and_get_column_const(c1.get()); @@ -221,6 +223,7 @@ class DecimalComparison { vector_constant(c0_vec->get_data(), b, vec_res, scale); else { LOG(FATAL) << "Wrong column in Decimal comparison"; + __builtin_unreachable(); } } else { if (const ColVecA* c0_vec = check_and_get_column(c0.get())) { @@ -229,9 +232,11 @@ class DecimalComparison { c1_vec->get_data(), vec_res, scale); else { LOG(FATAL) << "Wrong column in Decimal comparison"; + __builtin_unreachable(); } } else { LOG(FATAL) << "Wrong column in Decimal comparison"; + __builtin_unreachable(); } } return c_res; @@ -258,6 +263,7 @@ class DecimalComparison { if (overflow) { LOG(FATAL) << "Can't compare"; + __builtin_unreachable(); } } else { if constexpr (scale_left) x *= scale; diff --git a/be/src/vec/core/field.h b/be/src/vec/core/field.h index 6361df6e788f26..356216e70741d3 100644 --- a/be/src/vec/core/field.h +++ b/be/src/vec/core/field.h @@ -217,29 +217,37 @@ class JsonbField { bool operator<(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } bool operator<=(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } bool operator==(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } bool operator>(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } bool operator>=(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } bool operator!=(const JsonbField& r) const { LOG(FATAL) << "comparing between JsonbField is not supported"; + __builtin_unreachable(); } const JsonbField& operator+=(const JsonbField& r) { LOG(FATAL) << "Not support plus opration on JsonbField"; + __builtin_unreachable(); } const JsonbField& operator-=(const JsonbField& r) { LOG(FATAL) << "Not support minus opration on JsonbField"; + __builtin_unreachable(); } private: @@ -298,6 +306,7 @@ class DecimalField { const DecimalField& operator+=(const DecimalField& r) { if (scale != r.get_scale()) { LOG(FATAL) << "Add different decimal fields"; + __builtin_unreachable(); } dec += r.get_value(); return *this; @@ -306,6 +315,7 @@ class DecimalField { const DecimalField& operator-=(const DecimalField& r) { if (scale != r.get_scale()) { LOG(FATAL) << "Sub different decimal fields"; + __builtin_unreachable(); } dec -= r.get_value(); return *this; @@ -415,6 +425,7 @@ class Field { LOG(FATAL) << "type not supported, type=" << Types::to_string(which); break; } + __builtin_unreachable(); } }; diff --git a/be/src/vec/data_types/data_type_decimal.h b/be/src/vec/data_types/data_type_decimal.h index 8bd7f8b01537f7..bd82ae362908b1 100644 --- a/be/src/vec/data_types/data_type_decimal.h +++ b/be/src/vec/data_types/data_type_decimal.h @@ -282,6 +282,7 @@ class DataTypeDecimal final : public IDataType { T scale_factor_for(const DataTypeDecimal& x, bool) const { if (get_scale() < x.get_scale()) { LOG(FATAL) << "Decimal result's scale is less then argiment's one"; + __builtin_unreachable(); } UInt32 scale_delta = get_scale() - x.get_scale(); /// scale_delta >= 0 diff --git a/be/src/vec/data_types/data_type_nothing.cpp b/be/src/vec/data_types/data_type_nothing.cpp index 9bae6b22e870d7..bd0cb9ae04b6c4 100644 --- a/be/src/vec/data_types/data_type_nothing.cpp +++ b/be/src/vec/data_types/data_type_nothing.cpp @@ -32,11 +32,13 @@ MutableColumnPtr DataTypeNothing::create_column() const { char* DataTypeNothing::serialize(const IColumn& column, char* buf, int be_exec_version) const { LOG(FATAL) << "not support"; + __builtin_unreachable(); } const char* DataTypeNothing::deserialize(const char* buf, IColumn* column, int be_exec_version) const { LOG(FATAL) << "not support"; + __builtin_unreachable(); } bool DataTypeNothing::equals(const IDataType& rhs) const { diff --git a/be/src/vec/data_types/data_type_struct.cpp b/be/src/vec/data_types/data_type_struct.cpp index 1b9c516d1faa5f..8a28a0998ccdce 100644 --- a/be/src/vec/data_types/data_type_struct.cpp +++ b/be/src/vec/data_types/data_type_struct.cpp @@ -75,6 +75,7 @@ DataTypeStruct::DataTypeStruct(const DataTypes& elems_, const Strings& names_) size_t size = elems.size(); if (names.size() != size) { LOG(FATAL) << "Wrong number of names passed to constructor of DataTypeStruct"; + __builtin_unreachable(); } Status st = check_tuple_names(names); @@ -272,6 +273,7 @@ void add_element_safe(const DataTypes& elems, IColumn& column, F&& impl) { // This is not a logical error because it may work with // user-supplied data. LOG(FATAL) << "Cannot read a tuple because not all elements are present"; + __builtin_unreachable(); } } } catch (...) { @@ -342,6 +344,7 @@ size_t DataTypeStruct::get_position_by_name(const String& name) const { } } LOG(FATAL) << "Struct doesn't have element with name '" + name + "'"; + __builtin_unreachable(); } std::optional DataTypeStruct::try_get_position_by_name(const String& name) const { diff --git a/be/src/vec/data_types/data_type_struct.h b/be/src/vec/data_types/data_type_struct.h index 3638b0d110a76f..172e5ec3ff598e 100644 --- a/be/src/vec/data_types/data_type_struct.h +++ b/be/src/vec/data_types/data_type_struct.h @@ -97,6 +97,7 @@ class DataTypeStruct final : public IDataType { Field get_field(const TExprNode& node) const override { LOG(FATAL) << "Unimplemented get_field for struct"; + __builtin_unreachable(); } void insert_default_into(IColumn& column) const override; diff --git a/be/src/vec/data_types/serde/data_type_number_serde.h b/be/src/vec/data_types/serde/data_type_number_serde.h index 32cbfd5069e6f0..e7f3b7f4debdbd 100644 --- a/be/src/vec/data_types/serde/data_type_number_serde.h +++ b/be/src/vec/data_types/serde/data_type_number_serde.h @@ -312,6 +312,7 @@ Status DataTypeNumberSerDe::write_one_cell_to_json(const IColumn& column, result.SetDouble(data[row_num]); } else { LOG(FATAL) << "unknown column type " << column.get_name() << " for writing to jsonb"; + __builtin_unreachable(); } return Status::OK(); } diff --git a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h index 91f7a590648335..744a62165fb7fe 100644 --- a/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h +++ b/be/src/vec/exec/format/parquet/byte_array_dict_decoder.h @@ -17,32 +17,26 @@ #pragma once -#include - #include +#include #include #include #include #include "common/status.h" -#include "util/bit_util.h" -#include "vec/columns/column_string.h" #include "vec/columns/columns_number.h" #include "vec/common/string_ref.h" #include "vec/core/types.h" #include "vec/data_types/data_type.h" -#include "vec/exec/format/format_common.h" #include "vec/exec/format/parquet/decoder.h" #include "vec/exec/format/parquet/parquet_common.h" -namespace doris { -namespace vectorized { +namespace doris::vectorized { template class ColumnDecimal; -} // namespace vectorized -} // namespace doris - -namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; class ByteArrayDictDecoder final : public BaseDictDecoder { public: diff --git a/be/src/vec/exec/format/parquet/decoder.h b/be/src/vec/exec/format/parquet/decoder.h index 4c74c639c35bc7..57fecf4abfb7a2 100644 --- a/be/src/vec/exec/format/parquet/decoder.h +++ b/be/src/vec/exec/format/parquet/decoder.h @@ -19,20 +19,18 @@ #include #include -#include +#include #include #include #include #include #include "common/status.h" -#include "schema_desc.h" #include "util/rle_encoding.h" #include "util/slice.h" #include "vec/columns/column.h" #include "vec/columns/column_dictionary.h" -#include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" @@ -41,11 +39,14 @@ #include "vec/data_types/data_type.h" #include "vec/data_types/data_type_decimal.h" // IWYU pragma: keep #include "vec/data_types/data_type_nullable.h" -#include "vec/exec/format/format_common.h" #include "vec/exec/format/parquet/parquet_common.h" namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; + class Decoder { public: Decoder() = default; @@ -84,6 +85,7 @@ class Decoder { virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; + __builtin_unreachable(); } protected: @@ -95,16 +97,16 @@ class Decoder { class BaseDictDecoder : public Decoder { public: BaseDictDecoder() = default; - virtual ~BaseDictDecoder() override = default; + ~BaseDictDecoder() override = default; // Set the data to be decoded - virtual void set_data(Slice* data) override { + void set_data(Slice* data) override { _data = data; _offset = 0; uint8_t bit_width = *data->data; - _index_batch_decoder.reset( - new RleBatchDecoder(reinterpret_cast(data->data) + 1, - static_cast(data->size) - 1, bit_width)); + _index_batch_decoder = std::make_unique>( + reinterpret_cast(data->data) + 1, static_cast(data->size) - 1, + bit_width); } protected: @@ -125,7 +127,7 @@ class BaseDictDecoder : public Decoder { while (size_t run_length = select_vector.get_next_run(&read_type)) { switch (read_type) { case ColumnSelectVector::CONTENT: { - uint32_t* start_index = &_indexes[0]; + uint32_t* start_index = _indexes.data(); column_data.insert(start_index + dict_index, start_index + dict_index + run_length); dict_index += run_length; break; @@ -148,7 +150,7 @@ class BaseDictDecoder : public Decoder { Status skip_values(size_t num_values) override { _indexes.resize(num_values); - _index_batch_decoder->GetBatch(&_indexes[0], num_values); + _index_batch_decoder->GetBatch(_indexes.data(), num_values); return Status::OK(); } diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.h b/be/src/vec/exec/format/parquet/parquet_column_convert.h index cbbb3ac90a4d59..ede6e426488ecb 100644 --- a/be/src/vec/exec/format/parquet/parquet_column_convert.h +++ b/be/src/vec/exec/format/parquet/parquet_column_convert.h @@ -19,13 +19,13 @@ #include -#include "gutil/endian.h" #include "vec/core/types.h" #include "vec/data_types/data_type_factory.hpp" #include "vec/exec/format/column_type_convert.h" #include "vec/exec/format/format_common.h" #include "vec/exec/format/parquet/decoder.h" #include "vec/exec/format/parquet/parquet_common.h" +#include "vec/exec/format/parquet/schema_desc.h" namespace doris::vectorized::parquet { diff --git a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h index eb97ab0c885bf3..0ca6859ac83876 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_chunk_reader.h @@ -18,8 +18,8 @@ #pragma once #include -#include +#include #include #include #include @@ -45,13 +45,16 @@ namespace io { class BufferedStreamReader; struct IOContext; } // namespace io -namespace vectorized { -struct FieldSchema; -} // namespace vectorized + } // namespace doris namespace doris::vectorized { +struct FieldSchema; +template +class ColumnStr; +using ColumnString = ColumnStr; + /** * Read and decode parquet column data into doris block column. *

Usage:

@@ -91,7 +94,7 @@ class ColumnChunkReader { Status init(); // Whether the chunk reader has a more page to read. - bool has_next_page() { return _chunk_parsed_values < _metadata.num_values; } + bool has_next_page() const { return _chunk_parsed_values < _metadata.num_values; } // Deprecated // Seek to the specific page, page_header_offset must be the start offset of the page header. diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp index 6de7c06a7986f7..71b103d7fdb105 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.cpp @@ -40,15 +40,6 @@ #include "vec/exec/format/parquet/level_decoder.h" #include "vparquet_column_chunk_reader.h" -namespace cctz { -class time_zone; -} // namespace cctz -namespace doris { -namespace io { -struct IOContext; -} // namespace io -} // namespace doris - namespace doris::vectorized { static void fill_struct_null_map(FieldSchema* field, NullMap& null_map, diff --git a/be/src/vec/exec/format/parquet/vparquet_column_reader.h b/be/src/vec/exec/format/parquet/vparquet_column_reader.h index a3a2815e1af840..d12eac2f383d58 100644 --- a/be/src/vec/exec/format/parquet/vparquet_column_reader.h +++ b/be/src/vec/exec/format/parquet/vparquet_column_reader.h @@ -31,7 +31,6 @@ #include "io/fs/buffered_reader.h" #include "io/fs/file_reader_writer_fwd.h" #include "parquet_column_convert.h" -#include "vec/columns/column_string.h" #include "vec/columns/columns_number.h" #include "vec/data_types/data_type.h" #include "vec/exec/format/parquet/parquet_common.h" @@ -40,17 +39,18 @@ namespace cctz { class time_zone; } // namespace cctz -namespace doris { -namespace io { + +namespace doris::io { struct IOContext; -} // namespace io -namespace vectorized { -struct FieldSchema; -} // namespace vectorized -} // namespace doris +} // namespace doris::io namespace doris::vectorized { +struct FieldSchema; +template +class ColumnStr; +using ColumnString = ColumnStr; + class ParquetColumnReader { public: struct Statistics { @@ -127,6 +127,7 @@ class ParquetColumnReader { virtual MutableColumnPtr convert_dict_column_to_string_column(const ColumnInt32* dict_column) { LOG(FATAL) << "Method convert_dict_column_to_string_column is not supported"; + __builtin_unreachable(); } static Status create(io::FileReaderSPtr file, FieldSchema* field, diff --git a/be/src/vec/exec/format/table/iceberg_reader.h b/be/src/vec/exec/format/table/iceberg_reader.h index b5050666d6e683..04be8d53f24c73 100644 --- a/be/src/vec/exec/format/table/iceberg_reader.h +++ b/be/src/vec/exec/format/table/iceberg_reader.h @@ -17,9 +17,8 @@ #pragma once -#include -#include - +#include +#include #include #include #include @@ -32,7 +31,6 @@ #include "table_format_reader.h" #include "util/runtime_profile.h" #include "vec/columns/column_dictionary.h" -#include "vec/columns/column_string.h" namespace tparquet { class KeyValue; @@ -53,6 +51,9 @@ struct IOContext; struct TypeDescriptor; namespace vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; class Block; class GenericReader; class ShardedKVCache; @@ -125,7 +126,7 @@ class IcebergTableReader : public TableFormatReader { Status _gen_col_name_maps(std::vector parquet_meta_kv); void _gen_file_col_names(); void _gen_new_colname_to_value_range(); - std::string _delet_file_cache_key(const std::string& path) { return "delete_" + path; } + static std::string _delet_file_cache_key(const std::string& path) { return "delete_" + path; } RuntimeProfile* _profile; RuntimeState* _state; diff --git a/be/src/vec/exec/vdata_gen_scan_node.cpp b/be/src/vec/exec/vdata_gen_scan_node.cpp index 13d19921b03517..d6502be578b9f8 100644 --- a/be/src/vec/exec/vdata_gen_scan_node.cpp +++ b/be/src/vec/exec/vdata_gen_scan_node.cpp @@ -55,6 +55,7 @@ VDataGenFunctionScanNode::VDataGenFunctionScanNode(ObjectPool* pool, const TPlan break; default: LOG(FATAL) << "Unsupported function type"; + __builtin_unreachable(); } } diff --git a/be/src/vec/exec/vset_operation_node.cpp b/be/src/vec/exec/vset_operation_node.cpp index d1f4d8ac2c823e..c4802bee4cfd87 100644 --- a/be/src/vec/exec/vset_operation_node.cpp +++ b/be/src/vec/exec/vset_operation_node.cpp @@ -268,6 +268,7 @@ Status VSetOperationNode::pull(RuntimeState* state, Block* output_ state->batch_size(), eos); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *_hash_table_variants); @@ -322,6 +323,7 @@ Status VSetOperationNode::process_build_block(Block& block, Runtim st = hash_table_build_process(arg, _arena); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *_hash_table_variants); @@ -372,6 +374,7 @@ Status VSetOperationNode::sink_probe(RuntimeState* state, int chil return process_hashtable_ctx.mark_data_in_hashtable(arg); } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *_hash_table_variants)); @@ -558,6 +561,7 @@ void VSetOperationNode::refresh_hash_table() { } } else { LOG(FATAL) << "FATAL: uninited hash table"; + __builtin_unreachable(); } }, *_hash_table_variants); diff --git a/be/src/vec/exprs/table_function/vexplode_split.h b/be/src/vec/exprs/table_function/vexplode_split.h index 8223e739cba8da..0a7423254297fb 100644 --- a/be/src/vec/exprs/table_function/vexplode_split.h +++ b/be/src/vec/exprs/table_function/vexplode_split.h @@ -17,26 +17,23 @@ #pragma once -#include -#include - +#include +#include #include #include #include "common/status.h" -#include "vec/columns/column_string.h" #include "vec/common/string_ref.h" #include "vec/data_types/data_type.h" #include "vec/exprs/table_function/table_function.h" -namespace doris { -namespace vectorized { -class Block; -} // namespace vectorized -} // namespace doris - namespace doris::vectorized { +class Block; +template +class ColumnStr; +using ColumnString = ColumnStr; + class VExplodeSplitTableFunction final : public TableFunction { ENABLE_FACTORY_CREATOR(VExplodeSplitTableFunction); 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 9eaa71d08ae20a..00e4d766540709 100644 --- a/be/src/vec/functions/array/function_array_enumerate_uniq.cpp +++ b/be/src/vec/functions/array/function_array_enumerate_uniq.cpp @@ -81,6 +81,7 @@ class FunctionArrayEnumerateUniq : public IFunction { DataTypePtr get_return_type_impl(const DataTypes& arguments) const override { if (arguments.empty()) { LOG(FATAL) << "Incorrect number of arguments for array_enumerate_uniq function"; + __builtin_unreachable(); } bool is_nested_nullable = false; for (size_t i = 0; i < arguments.size(); ++i) { diff --git a/be/src/vec/functions/array/function_arrays_overlap.h b/be/src/vec/functions/array/function_arrays_overlap.h index b46c874f7f9917..682f99acb409a6 100644 --- a/be/src/vec/functions/array/function_arrays_overlap.h +++ b/be/src/vec/functions/array/function_arrays_overlap.h @@ -20,18 +20,14 @@ #include #include -#include #include #include -#include #include -#include #include #include "common/status.h" #include "vec/columns/column.h" #include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/assert_cast.h" @@ -57,6 +53,10 @@ struct DefaultHash; namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; + template struct OverlapSetImpl { using ElementNativeType = typename NativeType::Type; @@ -237,7 +237,7 @@ class FunctionArraysOverlap : public IFunction { } private: - Status _execute_nullable(const ColumnArrayExecutionData& data, UInt8* dst_nullmap_data) const { + static Status _execute_nullable(const ColumnArrayExecutionData& data, UInt8* dst_nullmap_data) { for (ssize_t row = 0; row < data.offsets_ptr->size(); ++row) { if (dst_nullmap_data[row]) { continue; diff --git a/be/src/vec/functions/function.h b/be/src/vec/functions/function.h index df5d8d60942e30..30a849b4724211 100644 --- a/be/src/vec/functions/function.h +++ b/be/src/vec/functions/function.h @@ -409,6 +409,7 @@ class IFunction : public std::enable_shared_from_this, FunctionBasePtr build_impl(const ColumnsWithTypeAndName& /*arguments*/, const DataTypePtr& /*return_type*/) const final { LOG(FATAL) << "build_impl is not implemented for IFunction"; + __builtin_unreachable(); return {}; } }; diff --git a/be/src/vec/functions/function_cast.h b/be/src/vec/functions/function_cast.h index bcf04e98bd61b3..74f76684fb2923 100644 --- a/be/src/vec/functions/function_cast.h +++ b/be/src/vec/functions/function_cast.h @@ -646,6 +646,7 @@ struct ConvertImplNumberToJsonb { writer.writeDouble(data[i]); } else { LOG(FATAL) << "unsupported type "; + __builtin_unreachable(); } column_string->insert_data(writer.getOutput()->getBuffer(), writer.getOutput()->getSize()); @@ -893,6 +894,7 @@ struct ConvertImplFromJsonb { } } else { LOG(FATAL) << "unsupported type "; + __builtin_unreachable(); } } diff --git a/be/src/vec/functions/function_jsonb.cpp b/be/src/vec/functions/function_jsonb.cpp index 317ba846832725..a5c98b643e99f3 100644 --- a/be/src/vec/functions/function_jsonb.cpp +++ b/be/src/vec/functions/function_jsonb.cpp @@ -790,6 +790,7 @@ struct JsonbExtractImpl { } } else { LOG(FATAL) << "unexpected type "; + __builtin_unreachable(); } } diff --git a/be/src/vec/functions/function_variadic_arguments.h b/be/src/vec/functions/function_variadic_arguments.h index 271586ee432a27..dab685f10ae253 100644 --- a/be/src/vec/functions/function_variadic_arguments.h +++ b/be/src/vec/functions/function_variadic_arguments.h @@ -43,6 +43,7 @@ class FunctionVariadicArgumentsBase : public IFunction { res = create_decimal(27, 9, true); if (!res) { LOG(FATAL) << "Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()"; + __builtin_unreachable(); } } else res = std::make_shared(); diff --git a/be/src/vec/functions/in.h b/be/src/vec/functions/in.h index ef6ffc520f8a78..955f44ec491060 100644 --- a/be/src/vec/functions/in.h +++ b/be/src/vec/functions/in.h @@ -19,10 +19,9 @@ #pragma once #include -#include -#include #include +#include #include #include #include @@ -37,7 +36,6 @@ #include "vec/columns/column.h" #include "vec/columns/column_const.h" #include "vec/columns/column_nullable.h" -#include "vec/columns/column_string.h" #include "vec/columns/column_vector.h" #include "vec/columns/columns_number.h" #include "vec/common/string_ref.h" @@ -52,6 +50,10 @@ namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; + struct InState { bool use_set = true; std::unique_ptr hybrid_set; @@ -135,7 +137,7 @@ class FunctionIn : public IFunction { Status execute_impl(FunctionContext* context, Block& block, const ColumnNumbers& arguments, size_t result, size_t input_rows_count) const override { - auto in_state = reinterpret_cast( + auto* in_state = reinterpret_cast( context->get_function_state(FunctionContext::FRAGMENT_LOCAL)); if (!in_state) { return Status::RuntimeError("funciton context for function '{}' must have Set;", @@ -154,12 +156,13 @@ class FunctionIn : public IFunction { if (in_state->use_set) { if (materialized_column->is_nullable()) { - auto* null_col_ptr = vectorized::check_and_get_column( - materialized_column); - auto& null_map = assert_cast( - null_col_ptr->get_null_map_column()) - .get_data(); - auto* nested_col_ptr = null_col_ptr->get_nested_column_ptr().get(); + const auto* null_col_ptr = + vectorized::check_and_get_column( + materialized_column); + const auto& null_map = assert_cast( + null_col_ptr->get_null_map_column()) + .get_data(); + const auto* nested_col_ptr = null_col_ptr->get_nested_column_ptr().get(); if (nested_col_ptr->is_column_string()) { const auto* column_string_ptr = diff --git a/be/src/vec/json/parse2column.h b/be/src/vec/json/parse2column.h index 9df36bef283197..260cdf37b83844 100644 --- a/be/src/vec/json/parse2column.h +++ b/be/src/vec/json/parse2column.h @@ -21,20 +21,19 @@ #include #include "vec/columns/column.h" -#include "vec/columns/column_string.h" #include "vec/common/string_ref.h" -namespace doris { -namespace vectorized { +namespace doris::vectorized { + class SimdJSONParser; enum class ExtractType; template class JSONDataParser; -} // namespace vectorized -} // namespace doris - -namespace doris::vectorized { +template +class ColumnStr; +using ColumnString = ColumnStr; using JsonParser = JSONDataParser; + // parse a batch of json strings into column object, throws doris::Execption when failed void parse_json_to_variant(IColumn& column, const ColumnString& raw_json_column); diff --git a/be/src/vec/jsonb/serialize.h b/be/src/vec/jsonb/serialize.h index c8f9e51d041d08..9025642cae33c9 100644 --- a/be/src/vec/jsonb/serialize.h +++ b/be/src/vec/jsonb/serialize.h @@ -16,8 +16,8 @@ // under the License. #pragma once -#include +#include #include #include "olap/tablet_schema.h" diff --git a/be/src/vec/olap/olap_data_convertor.h b/be/src/vec/olap/olap_data_convertor.h index 979b8d13fd7fe3..d6a721f9792d07 100644 --- a/be/src/vec/olap/olap_data_convertor.h +++ b/be/src/vec/olap/olap_data_convertor.h @@ -443,6 +443,7 @@ class OlapBlockDataConvertor { const void* get_data() const override { return _results.data(); }; const void* get_data_at(size_t offset) const override { LOG(FATAL) << "now not support get_data_at for OlapColumnDataConvertorArray"; + __builtin_unreachable(); }; Status convert_to_olap() override; @@ -472,6 +473,7 @@ class OlapBlockDataConvertor { const void* get_data() const override { return _results.data(); }; const void* get_data_at(size_t offset) const override { LOG(FATAL) << "now not support get_data_at for OlapColumnDataConvertorMap"; + __builtin_unreachable(); }; private: diff --git a/be/src/vec/runtime/vdatetime_value.cpp b/be/src/vec/runtime/vdatetime_value.cpp index 6b769c68f9dadf..fcdaedea1839de 100644 --- a/be/src/vec/runtime/vdatetime_value.cpp +++ b/be/src/vec/runtime/vdatetime_value.cpp @@ -3407,6 +3407,7 @@ void DateV2Value::set_time(uint8_t hour, uint8_t minute, uint8_t second, uint date_v2_value_.microsecond_ = microsecond; } else { LOG(FATAL) << "Invalid operation 'set_time' for date!"; + __builtin_unreachable(); } } @@ -3416,6 +3417,7 @@ void DateV2Value::set_microsecond(uint32_t microsecond) { date_v2_value_.microsecond_ = microsecond; } else { LOG(FATAL) << "Invalid operation 'set_microsecond' for date!"; + __builtin_unreachable(); } }