abort instead of __builtin_unreachable in debug builds

This commit is contained in:
Alexander Tokmakov 2022-10-07 21:20:14 +02:00
parent 26b2db2c94
commit 4175f8cde6
73 changed files with 116 additions and 107 deletions

View File

@ -123,11 +123,15 @@
/// - tries to print failed assertion into server log
/// It can be used for all assertions except heavy ones.
/// Heavy assertions (that run loops or call complex functions) are allowed in debug builds only.
/// Also it makes sense to call abort() instead of __builtin_unreachable() in debug builds,
/// because SIGABRT is easier to debug than SIGTRAP (the second one makes gdb crazy)
#if !defined(chassert)
#if defined(ABORT_ON_LOGICAL_ERROR)
#define chassert(x) static_cast<bool>(x) ? void(0) : abortOnFailedAssertion(#x)
#define UNREACHABLE() abort()
#else
#define chassert(x) ((void)0)
#define UNREACHABLE() __builtin_unreachable()
#endif
#endif

View File

@ -11,7 +11,7 @@
/// Thread sanitizer tries to do something on exit that we don't need if we want to exit immediately,
/// while connection handling threads are still run.
(void)syscall(SYS_exit_group, code);
__builtin_unreachable();
UNREACHABLE();
#else
_exit(code);
#endif

View File

@ -219,7 +219,7 @@ auto instructionFailToString(InstructionFail fail)
case InstructionFail::AVX512:
ret("AVX512");
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -145,7 +145,7 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String
{
e.addMessage("Could not parse " + file_path);
e.rethrow();
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -209,7 +209,7 @@ namespace
case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel();
case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel();
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -215,7 +215,7 @@ std::vector<UUID> IAccessStorage::insert(const std::vector<AccessEntityPtr> & mu
e.addMessage("After successfully inserting {}/{}: {}", successfully_inserted.size(), multiple_entities.size(), successfully_inserted_str);
}
e.rethrow();
__builtin_unreachable();
UNREACHABLE();
}
}
@ -319,7 +319,7 @@ std::vector<UUID> IAccessStorage::remove(const std::vector<UUID> & ids, bool thr
e.addMessage("After successfully removing {}/{}: {}", removed_names.size(), ids.size(), removed_names_str);
}
e.rethrow();
__builtin_unreachable();
UNREACHABLE();
}
}
@ -416,7 +416,7 @@ std::vector<UUID> IAccessStorage::update(const std::vector<UUID> & ids, const Up
e.addMessage("After successfully updating {}/{}: {}", names_of_updated.size(), ids.size(), names_of_updated_str);
}
e.rethrow();
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -56,7 +56,7 @@ static constexpr const char * getNameByTrait()
return "groupArraySample";
// else if (Trait::sampler == Sampler::DETERMINATOR) // TODO
__builtin_unreachable();
UNREACHABLE();
}
template <typename T>

View File

@ -395,7 +395,7 @@ public:
break;
return (i == events_size) ? base - i : unmatched_idx;
}
__builtin_unreachable();
UNREACHABLE();
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override

View File

@ -114,7 +114,7 @@ public:
return "covarSamp";
if constexpr (StatFunc::kind == StatisticsFunctionKind::corr)
return "corr";
__builtin_unreachable();
UNREACHABLE();
}
DataTypePtr getReturnType() const override

View File

@ -407,7 +407,7 @@ public:
return "sumWithOverflow";
else if constexpr (Type == AggregateFunctionTypeSumKahan)
return "sumKahan";
__builtin_unreachable();
UNREACHABLE();
}
explicit AggregateFunctionSum(const DataTypes & argument_types_)

View File

@ -131,7 +131,7 @@ namespace
return createAggregateFunctionWithHashType<20>(use_64_bit_hash, argument_types, params);
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -30,7 +30,7 @@ UInt8 getDayOfWeek(const cctz::civil_day & date)
case cctz::weekday::saturday: return 6;
case cctz::weekday::sunday: return 7;
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -119,7 +119,7 @@ inline UInt32 updateWeakHash32(const DB::UInt8 * pos, size_t size, DB::UInt32 up
__builtin_memcpy(&value, pos, 7);
break;
default:
__builtin_unreachable();
UNREACHABLE();
}
reinterpret_cast<unsigned char *>(&value)[7] = size;
@ -194,8 +194,7 @@ inline size_t DefaultHash64(T key)
static_cast<UInt64>(key >> 128) ^
static_cast<UInt64>(key >> 256));
}
assert(false);
__builtin_unreachable();
UNREACHABLE();
}
template <typename T>
@ -454,8 +453,7 @@ struct IntHash32
return intHash32<salt>(u.out);
}
assert(false);
__builtin_unreachable();
UNREACHABLE();
}
};

View File

@ -353,11 +353,11 @@ struct HashTableFixedGrower
size_t bufSize() const { return 1ULL << key_bits; }
size_t place(size_t x) const { return x; }
/// You could write __builtin_unreachable(), but the compiler does not optimize everything, and it turns out less efficiently.
/// You could write UNREACHABLE(), but the compiler does not optimize everything, and it turns out less efficiently.
size_t next(size_t pos) const { return pos + 1; }
bool overflow(size_t /*elems*/) const { return false; }
void increaseSize() { __builtin_unreachable(); }
void increaseSize() { UNREACHABLE(); }
void set(size_t /*num_elems*/) {}
void setBufSize(size_t /*buf_size_*/) {}
};

View File

@ -26,7 +26,7 @@ Int32 IntervalKind::toAvgSeconds() const
case IntervalKind::Quarter: return 7889238; /// Exactly 1/4 of a year.
case IntervalKind::Year: return 31556952; /// The average length of a Gregorian year is equal to 365.2425 days
}
__builtin_unreachable();
UNREACHABLE();
}
Float64 IntervalKind::toSeconds() const
@ -52,7 +52,7 @@ Float64 IntervalKind::toSeconds() const
default:
throw Exception("Not possible to get precise number of seconds in non-precise interval", ErrorCodes::BAD_ARGUMENTS);
}
__builtin_unreachable();
UNREACHABLE();
}
bool IntervalKind::isFixedLength() const
@ -71,7 +71,7 @@ bool IntervalKind::isFixedLength() const
case IntervalKind::Quarter:
case IntervalKind::Year: return false;
}
__builtin_unreachable();
UNREACHABLE();
}
IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds)
@ -113,7 +113,7 @@ const char * IntervalKind::toKeyword() const
case IntervalKind::Quarter: return "QUARTER";
case IntervalKind::Year: return "YEAR";
}
__builtin_unreachable();
UNREACHABLE();
}
@ -133,7 +133,7 @@ const char * IntervalKind::toLowercasedKeyword() const
case IntervalKind::Quarter: return "quarter";
case IntervalKind::Year: return "year";
}
__builtin_unreachable();
UNREACHABLE();
}
@ -164,7 +164,7 @@ const char * IntervalKind::toDateDiffUnit() const
case IntervalKind::Year:
return "year";
}
__builtin_unreachable();
UNREACHABLE();
}
@ -195,7 +195,7 @@ const char * IntervalKind::toNameOfFunctionToIntervalDataType() const
case IntervalKind::Year:
return "toIntervalYear";
}
__builtin_unreachable();
UNREACHABLE();
}
@ -229,7 +229,7 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
case IntervalKind::Year:
return "toYear";
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -1,3 +1,4 @@
#include <base/defines.h>
#include <Common/TargetSpecific.h>
#include <Common/CpuId.h>
@ -42,7 +43,7 @@ String toString(TargetArch arch)
case TargetArch::AVX512VBMI: return "avx512vbmi";
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -76,7 +76,7 @@ const char * TasksStatsCounters::metricsProviderString(MetricsProvider provider)
case MetricsProvider::Netlink:
return "netlink";
}
__builtin_unreachable();
UNREACHABLE();
}
bool TasksStatsCounters::checkIfAvailable()

View File

@ -112,7 +112,7 @@ const char * errorMessage(Error code)
case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored";
}
__builtin_unreachable();
UNREACHABLE();
}
bool isHardwareError(Error zk_return_code)

View File

@ -36,7 +36,7 @@ static bool check()
return true;
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -393,7 +393,7 @@ void CompressionCodecDeflateQpl::doDecompressData(const char * source, UInt32 so
sw_codec->doDecompressData(source, source_size, dest, uncompressed_size);
return;
}
__builtin_unreachable();
UNREACHABLE();
}
void CompressionCodecDeflateQpl::flushAsynchronousDecompressRequests()

View File

@ -164,7 +164,7 @@ inline Int64 getMaxValueForByteSize(Int8 byte_size)
default:
assert(false && "only 1, 2, 4 and 8 data sizes are supported");
}
__builtin_unreachable();
UNREACHABLE();
}
struct WriteSpec

View File

@ -67,7 +67,7 @@ bool lessOp(A a, B b)
static_assert(is_integer<A> || std::is_floating_point_v<A>);
static_assert(is_integer<B> || std::is_floating_point_v<B>);
__builtin_unreachable();
UNREACHABLE();
}
template <typename A, typename B>

View File

@ -492,7 +492,7 @@ Field Field::restoreFromDump(std::string_view dump_)
}
show_error();
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -604,7 +604,7 @@ public:
case Types::AggregateFunctionState: return f(field.template get<AggregateFunctionStateData>());
}
__builtin_unreachable();
UNREACHABLE();
}
String dump() const;

View File

@ -36,7 +36,7 @@ String ISerialization::kindToString(Kind kind)
case Kind::SPARSE:
return "Sparse";
}
__builtin_unreachable();
UNREACHABLE();
}
ISerialization::Kind ISerialization::stringToKind(const String & str)

View File

@ -357,8 +357,7 @@ bool DatabaseReplicated::createDatabaseNodesInZooKeeper(const zkutil::ZooKeeperP
/// Other codes are unexpected, will throw
zkutil::KeeperMultiException::check(res, ops, responses);
chassert(false);
__builtin_unreachable();
UNREACHABLE();
}
bool DatabaseReplicated::looksLikeReplicatedDatabasePath(const ZooKeeperPtr & current_zookeeper, const String & path)

View File

@ -124,7 +124,7 @@ namespace DB
return "none";
}
__builtin_unreachable();
UNREACHABLE();
}
QueryPipeline RedisDictionarySource::loadAll()

View File

@ -1,5 +1,6 @@
#pragma once
#include <base/defines.h>
#include <base/types.h>
namespace DB
@ -32,7 +33,7 @@ inline String toString(DataSourceType data_source_type)
case DataSourceType::AzureBlobStorage:
return "azure_blob_storage";
}
__builtin_unreachable();
UNREACHABLE();
}
struct DataSourceDescription

View File

@ -127,7 +127,7 @@ private:
case ReadType::REMOTE_FS_READ_AND_PUT_IN_CACHE:
return "REMOTE_FS_READ_AND_PUT_IN_CACHE";
}
__builtin_unreachable();
UNREACHABLE();
}
size_t first_offset = 0;

View File

@ -1,3 +1,4 @@
#include <base/defines.h>
#include <Disks/ObjectStorages/MetadataFromDiskTransactionState.h>
namespace DB
@ -16,7 +17,7 @@ std::string toString(MetadataFromDiskTransactionState state)
case MetadataFromDiskTransactionState::PARTIALLY_ROLLED_BACK:
return "PARTIALLY_ROLLED_BACK";
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -97,7 +97,7 @@ DiskPtr VolumeJBOD::getDisk(size_t /* index */) const
return disks_by_size.top().disk;
}
}
__builtin_unreachable();
UNREACHABLE();
}
ReservationPtr VolumeJBOD::reserve(UInt64 bytes)
@ -137,7 +137,7 @@ ReservationPtr VolumeJBOD::reserve(UInt64 bytes)
return reservation;
}
}
__builtin_unreachable();
UNREACHABLE();
}
bool VolumeJBOD::areMergesAvoided() const

View File

@ -70,7 +70,7 @@ String escapingRuleToString(FormatSettings::EscapingRule escaping_rule)
case FormatSettings::EscapingRule::Raw:
return "Raw";
}
__builtin_unreachable();
UNREACHABLE();
}
void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
@ -99,7 +99,7 @@ void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule esca
readStringInto(out, buf);
break;
default:
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -3535,7 +3535,7 @@ namespace
}
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -2273,7 +2273,7 @@ struct ToNumberMonotonicity
}
}
__builtin_unreachable();
UNREACHABLE();
}
};

View File

@ -151,7 +151,7 @@ struct IntegerRoundingComputation
}
}
__builtin_unreachable();
UNREACHABLE();
}
static ALWAYS_INLINE T compute(T x, T scale)
@ -165,7 +165,7 @@ struct IntegerRoundingComputation
return computeImpl(x, scale);
}
__builtin_unreachable();
UNREACHABLE();
}
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral<T>
@ -249,7 +249,7 @@ inline float roundWithMode(float x, RoundingMode mode)
case RoundingMode::Trunc: return truncf(x);
}
__builtin_unreachable();
UNREACHABLE();
}
inline double roundWithMode(double x, RoundingMode mode)
@ -262,7 +262,7 @@ inline double roundWithMode(double x, RoundingMode mode)
case RoundingMode::Trunc: return trunc(x);
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename T>

View File

@ -194,7 +194,7 @@ struct TimeWindowImpl<TUMBLE>
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename ToType, IntervalKind::Kind unit>
@ -397,7 +397,7 @@ struct TimeWindowImpl<HOP>
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename ToType, IntervalKind::Kind kind>
@ -546,7 +546,7 @@ struct TimeWindowImpl<WINDOW_ID>
default:
throw Exception("Fraction seconds are unsupported by windows yet", ErrorCodes::SYNTAX_ERROR);
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename ToType, IntervalKind::Kind kind>

View File

@ -384,7 +384,7 @@ bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, Coordina
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -510,7 +510,7 @@ private:
return execute<FromDataType, DataTypeDate, IntervalKind::Year>(from, time_column, num_units, result_type, time_zone, scale);
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename FromDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>

View File

@ -52,7 +52,7 @@ std::string toContentEncodingName(CompressionMethod method)
case CompressionMethod::None:
return "";
}
__builtin_unreachable();
UNREACHABLE();
}
CompressionMethod chooseHTTPCompressionMethod(const std::string & list)

View File

@ -5,6 +5,7 @@
#pragma clang diagnostic ignored "-Wdouble-promotion"
#endif
#include <base/defines.h>
#include <double-conversion/double-conversion.h>
#include <boost/noncopyable.hpp>

View File

@ -85,7 +85,7 @@ public:
case Status::NEEDS_MORE_INPUT:
return "NEEDS_MORE_INPUT";
}
__builtin_unreachable();
UNREACHABLE();
}
explicit HadoopSnappyReadBuffer(

View File

@ -206,7 +206,7 @@ DB::AggregatedDataVariants::Type convertToTwoLevelTypeIfPossible(DB::AggregatedD
default:
return type;
}
__builtin_unreachable();
UNREACHABLE();
}
void initDataVariantsWithSizeHint(

View File

@ -688,7 +688,7 @@ struct AggregatedDataVariants : private boost::noncopyable
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
/// The size without taking into account the row in which data is written for the calculation of TOTALS.
@ -705,7 +705,7 @@ struct AggregatedDataVariants : private boost::noncopyable
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
const char * getMethodName() const
@ -721,7 +721,7 @@ struct AggregatedDataVariants : private boost::noncopyable
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
bool isTwoLevel() const
@ -737,7 +737,7 @@ struct AggregatedDataVariants : private boost::noncopyable
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
#define APPLY_FOR_VARIANTS_CONVERTIBLE_TO_TWO_LEVEL(M) \

View File

@ -653,7 +653,7 @@ String FileSegment::stateToString(FileSegment::State state)
case FileSegment::State::SKIP_CACHE:
return "SKIP_CACHE";
}
__builtin_unreachable();
UNREACHABLE();
}
void FileSegment::assertCorrectness() const

View File

@ -156,7 +156,7 @@ ComparisonGraph::CompareResult ComparisonGraph::pathToCompareResult(Path path, b
case Path::GREATER: return inverse ? CompareResult::LESS : CompareResult::GREATER;
case Path::GREATER_OR_EQUAL: return inverse ? CompareResult::LESS_OR_EQUAL : CompareResult::GREATER_OR_EQUAL;
}
__builtin_unreachable();
UNREACHABLE();
}
std::optional<ComparisonGraph::Path> ComparisonGraph::findPath(size_t start, size_t finish) const

View File

@ -23,7 +23,7 @@ static String typeToString(FilesystemCacheLogElement::CacheType type)
case FilesystemCacheLogElement::CacheType::WRITE_THROUGH_CACHE:
return "WRITE_THROUGH_CACHE";
}
__builtin_unreachable();
UNREACHABLE();
}
NamesAndTypesList FilesystemCacheLogElement::getNamesAndTypes()

View File

@ -624,7 +624,7 @@ namespace
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
}
@ -1852,7 +1852,7 @@ private:
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type) ;
}
__builtin_unreachable();
UNREACHABLE();
}
template <JoinStrictness STRICTNESS, typename Map>

View File

@ -279,7 +279,7 @@ public:
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
size_t getTotalByteCountImpl(Type which) const
@ -295,7 +295,7 @@ public:
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
size_t getBufferSizeInCells(Type which) const
@ -311,7 +311,7 @@ public:
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
};

View File

@ -32,8 +32,7 @@ BlockIO InterpreterTransactionControlQuery::execute()
case ASTTransactionControl::SET_SNAPSHOT:
return executeSetSnapshot(session_context, tcl.snapshot);
}
assert(false);
__builtin_unreachable();
UNREACHABLE();
}
BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context)

View File

@ -37,7 +37,7 @@ void callWithType(TypeIndex type, F && f)
DISPATCH(DateTime64)
#undef DISPATCH
__builtin_unreachable();
UNREACHABLE();
}
template <typename TKey, ASOFJoinInequality inequality>

View File

@ -42,7 +42,7 @@ size_t SetVariantsTemplate<Variant>::getTotalRowCount() const
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename Variant>
@ -58,7 +58,7 @@ size_t SetVariantsTemplate<Variant>::getTotalByteCount() const
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}
template <typename Variant>

View File

@ -124,7 +124,7 @@ const char * ASTAlterCommand::typeToString(ASTAlterCommand::Type type)
case MODIFY_DATABASE_SETTING: return "MODIFY_DATABASE_SETTING";
case MODIFY_COMMENT: return "MODIFY_COMMENT";
}
__builtin_unreachable();
UNREACHABLE();
}
void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const

View File

@ -116,7 +116,7 @@ private:
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
}
__builtin_unreachable();
UNREACHABLE();
}
};

View File

@ -1,3 +1,4 @@
#include <base/defines.h>
#include <Parsers/Lexer.h>
#include <Common/StringUtils/StringUtils.h>
#include <base/find_symbols.h>
@ -39,7 +40,7 @@ Token quotedString(const char *& pos, const char * const token_begin, const char
continue;
}
__builtin_unreachable();
UNREACHABLE();
}
}
@ -414,7 +415,7 @@ APPLY_FOR_TOKENS(M)
#undef M
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -512,7 +512,7 @@ DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack extension type {:x} is not supported", object_ext.type());
}
}
__builtin_unreachable();
UNREACHABLE();
}
DataTypes MsgPackSchemaReader::readRowAndGetDataTypes()

View File

@ -37,7 +37,7 @@ std::string IProcessor::statusToName(Status status)
return "ExpandPipeline";
}
__builtin_unreachable();
UNREACHABLE();
}
}

View File

@ -639,7 +639,7 @@ static void addMergingFinal(
sort_description, max_block_size, merging_params.graphite_params, now);
}
__builtin_unreachable();
UNREACHABLE();
};
pipe.addTransform(get_merging_processor());
@ -1240,7 +1240,7 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type)
return "Skip";
}
__builtin_unreachable();
UNREACHABLE();
}
static const char * readTypeToString(ReadFromMergeTree::ReadType type)
@ -1255,7 +1255,7 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type)
return "InReverseOrder";
}
__builtin_unreachable();
UNREACHABLE();
}
void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const

View File

@ -88,7 +88,7 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr
return "after_having_auto threshold " + std::to_string(auto_include_threshold);
}
__builtin_unreachable();
UNREACHABLE();
}
void TotalsHavingStep::describeActions(FormatSettings & settings) const

View File

@ -45,7 +45,7 @@ static FillColumnDescription::StepFunction getStepFunction(
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
}
__builtin_unreachable();
UNREACHABLE();
}
static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type)

View File

@ -744,7 +744,7 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St
return std::current_exception();
}
__builtin_unreachable();
UNREACHABLE();
}
void FinalizingViewsTransform::work()

View File

@ -390,7 +390,7 @@ namespace
case CALL_WITH_STREAM_IO: return "ExecuteQueryWithStreamIO()";
case CALL_MAX: break;
}
__builtin_unreachable();
UNREACHABLE();
}
bool isInputStreaming(CallType call_type)
@ -550,7 +550,7 @@ namespace
case CALL_WITH_STREAM_IO: return std::make_unique<Responder<CALL_WITH_STREAM_IO>>();
case CALL_MAX: break;
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -878,8 +878,7 @@ try
}
else
{
assert(false);
__builtin_unreachable();
UNREACHABLE();
}
used_output.finalize();

View File

@ -490,7 +490,7 @@ static GetColumnsOptions::Kind defaultKindToGetKind(ColumnDefaultKind kind)
case ColumnDefaultKind::Ephemeral:
return GetColumnsOptions::Ephemeral;
}
__builtin_unreachable();
UNREACHABLE();
}
NamesAndTypesList ColumnsDescription::getByNames(const GetColumnsOptions & options, const Names & names) const

View File

@ -90,7 +90,7 @@ String BackgroundJobsAssignee::toString(Type type)
case Type::Moving:
return "Moving";
}
__builtin_unreachable();
UNREACHABLE();
}
void BackgroundJobsAssignee::start()

View File

@ -2581,7 +2581,7 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri
return "true";
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -928,7 +928,7 @@ String MergeTreeData::MergingParams::getModeName() const
case VersionedCollapsing: return "VersionedCollapsing";
}
__builtin_unreachable();
UNREACHABLE();
}
Int64 MergeTreeData::getMaxBlockNumber() const

View File

@ -244,7 +244,7 @@ Block MergeTreeDataWriter::mergeBlock(
block, 1, sort_description, block_size + 1, merging_params.graphite_params, time(nullptr));
}
__builtin_unreachable();
UNREACHABLE();
};
auto merging_algorithm = get_merging_algorithm();

View File

@ -125,7 +125,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa
}
}
__builtin_unreachable();
UNREACHABLE();
}
PartitionReadResponse ParallelReplicasReadingCoordinator::handleRequest(PartitionReadRequest request)

View File

@ -609,7 +609,7 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st
}
}
__builtin_unreachable();
UNREACHABLE();
}
void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk)

View File

@ -123,7 +123,7 @@ bool ReplicatedMergeTreeRestartingThread::runImpl()
}
else
{
__builtin_unreachable();
UNREACHABLE();
}
try

View File

@ -1422,7 +1422,7 @@ size_t StorageDistributed::getRandomShardIndex(const Cluster::ShardsInfo & shard
res -= shards[i].weight;
}
__builtin_unreachable();
UNREACHABLE();
}

View File

@ -296,7 +296,7 @@ namespace
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
__builtin_unreachable();
UNREACHABLE();
}
class AddingAggregatedChunkInfoTransform : public ISimpleTransform
@ -895,7 +895,7 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec)
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
__builtin_unreachable();
UNREACHABLE();
}
UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec)
@ -923,7 +923,7 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec)
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
__builtin_unreachable();
UNREACHABLE();
}
void StorageWindowView::addFireSignal(std::set<UInt32> & signals)

View File

@ -352,3 +352,8 @@ find $ROOT_PATH | sort -f | uniq -i -c | awk '{ if ($1 > 1) print }'
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '::(is|read)_symlink' | grep -v "STYLE_CHECK_ALLOW_STD_FS_SYMLINK" && echo "Use DB::FS::isSymlink and DB::FS::readSymlink instead"
# Forbid __builtin_unreachable(), because it's hard to debug when it becomes reachable
find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
grep -vP $EXCLUDE_DIRS |
xargs grep -P '__builtin_unreachable' && echo "Use UNREACHABLE() from defines.h instead"