Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions be/src/exec/rowid_fetcher.h
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,11 @@
#include <gen_cpp/internal_service.pb.h>

#include <memory>
#include <utility>
#include <vector>

#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"

Expand All @@ -38,6 +37,9 @@ class RuntimeState;
class TupleDescriptor;

namespace vectorized {
template <typename T>
class ColumnStr;
using ColumnString = ColumnStr<UInt32>;
class MutableBlock;
} // namespace vectorized

Expand All @@ -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);

Expand Down
4 changes: 4 additions & 0 deletions be/src/exprs/hybrid_set.h
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
3 changes: 3 additions & 0 deletions be/src/olap/comparison_predicate.h
Original file line number Diff line number Diff line change
Expand Up @@ -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 =
Expand Down Expand Up @@ -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 =
Expand Down Expand Up @@ -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 =
Expand Down
2 changes: 2 additions & 0 deletions be/src/olap/in_list_predicate.h
Original file line number Diff line number Diff line change
Expand Up @@ -427,6 +427,7 @@ class InListPredicateBase : public ColumnPredicate {
}
} else {
LOG(FATAL) << "column_dictionary must use StringRef predicate.";
__builtin_unreachable();
}
} else {
auto& pred_col =
Expand Down Expand Up @@ -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<
Expand Down
2 changes: 2 additions & 0 deletions be/src/olap/like_column_predicate.h
Original file line number Diff line number Diff line change
Expand Up @@ -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()) {
Expand All @@ -153,6 +154,7 @@ class LikeColumnPredicate : public ColumnPredicate {
}
} else {
LOG(FATAL) << "vectorized (not) like predicates should be dict column";
__builtin_unreachable();
}
}
}
Expand Down
1 change: 1 addition & 0 deletions be/src/olap/match_predicate.h
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions be/src/olap/page_cache.h
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,7 @@ class StoragePageCache {
}
default:
LOG(FATAL) << "get error type page cache";
__builtin_unreachable();
}
LOG(FATAL) << "__builtin_unreachable";
__builtin_unreachable();
Expand Down
2 changes: 2 additions & 0 deletions be/src/olap/rowset/segment_v2/hierarchical_data_reader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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) {
Expand Down Expand Up @@ -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) {
Expand Down
1 change: 1 addition & 0 deletions be/src/pipeline/exec/exchange_sink_buffer.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -440,6 +440,7 @@ void ExchangeSinkBuffer<Parent>::_ended(InstanceLoId id) {
LOG(INFO) << ss.str();

LOG(FATAL) << "not find the instance id";
__builtin_unreachable();
} else {
std::unique_lock<std::mutex> lock(*_instance_to_package_queue_mutex[id]);
if (!_rpc_channel_is_idle[id]) {
Expand Down
1 change: 1 addition & 0 deletions be/src/pipeline/exec/exchange_sink_buffer.h
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,7 @@ class ExchangeSendCallback : public ::doris::DummyBrpcCallback<Response> {
LOG(FATAL) << "brpc callback error: " << exp.what();
} catch (...) {
LOG(FATAL) << "brpc callback error.";
__builtin_unreachable();
}
}
int64_t start_rpc_time;
Expand Down
2 changes: 2 additions & 0 deletions be/src/pipeline/exec/set_probe_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,7 @@ Status SetProbeSinkOperatorX<is_intersect>::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));
Expand Down Expand Up @@ -283,6 +284,7 @@ void SetProbeSinkOperatorX<is_intersect>::_refresh_hash_table(
}
} else {
LOG(FATAL) << "FATAL: uninited hash table";
__builtin_unreachable();
}
},
*hash_table_variants);
Expand Down
1 change: 1 addition & 0 deletions be/src/pipeline/exec/set_sink_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,7 @@ Status SetSinkOperatorX<is_intersect>::_process_build_block(
static_cast<void>(hash_table_build_process(arg, local_state._arena));
} else {
LOG(FATAL) << "FATAL: uninited hash table";
__builtin_unreachable();
}
},
*local_state._shared_state->hash_table_variants);
Expand Down
1 change: 1 addition & 0 deletions be/src/pipeline/exec/set_source_operator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -102,6 +102,7 @@ Status SetSourceOperatorX<is_intersect>::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);
Expand Down
11 changes: 11 additions & 0 deletions be/src/runtime/jsonb_value.h
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
10 changes: 7 additions & 3 deletions be/src/runtime/primitive_type.h
Original file line number Diff line number Diff line change
Expand Up @@ -20,15 +20,13 @@
#include <gen_cpp/Opcodes_types.h>
#include <gen_cpp/Types_types.h>
#include <glog/logging.h>
#include <stdint.h>

#include <cstdint>
#include <string>
#include <type_traits>

#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"
Expand All @@ -37,6 +35,12 @@

namespace doris {

namespace vectorized {
template <typename T>
class ColumnStr;
using ColumnString = ColumnStr<UInt32>;
} // namespace vectorized

class DecimalV2Value;
struct StringRef;
struct JsonBinaryValue;
Expand Down
1 change: 1 addition & 0 deletions be/src/runtime/snapshot_loader.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 1 addition & 0 deletions be/src/util/easy_json.cc
Original file line number Diff line number Diff line change
Expand Up @@ -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_);
Expand Down
1 change: 1 addition & 0 deletions be/src/util/timezone_utils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,10 @@
#pragma once

#include <glog/logging.h>
#include <stddef.h>
#include <stdint.h>

#include <algorithm>
#include <boost/iterator/iterator_facade.hpp>
#include <cstddef>
#include <cstdint>
#include <memory>
#include <string>
#include <string_view>
Expand All @@ -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 <typename T>
class ColumnStr;
using ColumnString = ColumnStr<UInt32>;

template <typename T>
struct AggOrthBitmapBaseData {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
}
};

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
4 changes: 4 additions & 0 deletions be/src/vec/aggregate_functions/aggregate_function_window.h
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
Loading