Merge pull request #64435 from rschu1ze/revert-revert-64035-unreachable-unreachable

Revert "Revert "Remove some unnecessary UNREACHABLEs""
This commit is contained in:
Robert Schulze 2024-05-30 10:42:40 +00:00 committed by GitHub
commit d776d88797
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
49 changed files with 30 additions and 119 deletions

View File

@ -10,6 +10,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int KEEPER_EXCEPTION;
}
@ -441,7 +442,7 @@ void ReconfigCommand::execute(const DB::ASTKeeperQuery * query, DB::KeeperClient
new_members = query->args[1].safeGet<String>();
break;
default:
UNREACHABLE();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected operation: {}", operation);
}
auto response = client->zookeeper->reconfig(joining, leaving, new_members);

View File

@ -155,8 +155,8 @@ auto instructionFailToString(InstructionFail fail)
ret("AVX2");
case InstructionFail::AVX512:
ret("AVX512");
#undef ret
}
UNREACHABLE();
}

View File

@ -144,8 +144,7 @@ AccessEntityPtr deserializeAccessEntity(const String & definition, const String
catch (Exception & e)
{
e.addMessage("Could not parse " + file_path);
e.rethrow();
UNREACHABLE();
throw;
}
}

View File

@ -258,7 +258,7 @@ namespace
case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel();
case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel();
}
UNREACHABLE();
chassert(false);
}
}

View File

@ -257,8 +257,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();
UNREACHABLE();
throw;
}
}
@ -361,8 +360,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();
UNREACHABLE();
throw;
}
}
@ -458,8 +456,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();
UNREACHABLE();
throw;
}
}

View File

@ -60,14 +60,13 @@ struct GroupArrayTrait
template <typename Trait>
constexpr const char * getNameByTrait()
{
if (Trait::last)
if constexpr (Trait::last)
return "groupArrayLast";
if (Trait::sampler == Sampler::NONE)
return "groupArray";
else if (Trait::sampler == Sampler::RNG)
return "groupArraySample";
UNREACHABLE();
switch (Trait::sampler)
{
case Sampler::NONE: return "groupArray";
case Sampler::RNG: return "groupArraySample";
}
}
template <typename T>

View File

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

View File

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

View File

@ -41,7 +41,6 @@ UInt8 getDayOfWeek(const cctz::civil_day & date)
case cctz::weekday::saturday: return 6;
case cctz::weekday::sunday: return 7;
}
UNREACHABLE();
}
inline cctz::time_point<cctz::seconds> lookupTz(const cctz::time_zone & cctz_time_zone, const cctz::civil_day & date)

View File

@ -34,8 +34,6 @@ Int64 IntervalKind::toAvgNanoseconds() const
default:
return toAvgSeconds() * NANOSECONDS_PER_SECOND;
}
UNREACHABLE();
}
Int32 IntervalKind::toAvgSeconds() const
@ -54,7 +52,6 @@ Int32 IntervalKind::toAvgSeconds() const
case IntervalKind::Kind::Quarter: return 7889238; /// Exactly 1/4 of a year.
case IntervalKind::Kind::Year: return 31556952; /// The average length of a Gregorian year is equal to 365.2425 days
}
UNREACHABLE();
}
Float64 IntervalKind::toSeconds() const
@ -80,7 +77,6 @@ Float64 IntervalKind::toSeconds() const
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not possible to get precise number of seconds in non-precise interval");
}
UNREACHABLE();
}
bool IntervalKind::isFixedLength() const
@ -99,7 +95,6 @@ bool IntervalKind::isFixedLength() const
case IntervalKind::Kind::Quarter:
case IntervalKind::Kind::Year: return false;
}
UNREACHABLE();
}
IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds)
@ -141,7 +136,6 @@ const char * IntervalKind::toKeyword() const
case IntervalKind::Kind::Quarter: return "QUARTER";
case IntervalKind::Kind::Year: return "YEAR";
}
UNREACHABLE();
}
@ -161,7 +155,6 @@ const char * IntervalKind::toLowercasedKeyword() const
case IntervalKind::Kind::Quarter: return "quarter";
case IntervalKind::Kind::Year: return "year";
}
UNREACHABLE();
}
@ -192,7 +185,6 @@ const char * IntervalKind::toDateDiffUnit() const
case IntervalKind::Kind::Year:
return "year";
}
UNREACHABLE();
}
@ -223,7 +215,6 @@ const char * IntervalKind::toNameOfFunctionToIntervalDataType() const
case IntervalKind::Kind::Year:
return "toIntervalYear";
}
UNREACHABLE();
}
@ -257,7 +248,6 @@ const char * IntervalKind::toNameOfFunctionExtractTimePart() const
case IntervalKind::Kind::Year:
return "toYear";
}
UNREACHABLE();
}

View File

@ -54,8 +54,6 @@ String toString(TargetArch arch)
case TargetArch::AMXTILE: return "amxtile";
case TargetArch::AMXINT8: return "amxint8";
}
UNREACHABLE();
}
}

View File

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

View File

@ -146,8 +146,6 @@ const char * errorMessage(Error code)
case Error::ZSESSIONMOVED: return "Session moved to another server, so operation is ignored";
case Error::ZNOTREADONLY: return "State-changing request is passed to read-only server";
}
UNREACHABLE();
}
bool isHardwareError(Error zk_return_code)

View File

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

View File

@ -21,6 +21,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/** NOTE DoubleDelta is surprisingly bad name. The only excuse is that it comes from an academic paper.
* Most people will think that "double delta" is just applying delta transform twice.
* But in fact it is something more than applying delta transform twice.
@ -142,9 +147,9 @@ namespace ErrorCodes
{
extern const int CANNOT_COMPRESS;
extern const int CANNOT_DECOMPRESS;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_SYNTAX_FOR_CODEC_TYPE;
extern const int ILLEGAL_CODEC_PARAMETER;
extern const int LOGICAL_ERROR;
}
namespace
@ -163,9 +168,8 @@ inline Int64 getMaxValueForByteSize(Int8 byte_size)
case sizeof(UInt64):
return std::numeric_limits<Int64>::max();
default:
assert(false && "only 1, 2, 4 and 8 data sizes are supported");
throw Exception(ErrorCodes::LOGICAL_ERROR, "only 1, 2, 4 and 8 data sizes are supported");
}
UNREACHABLE();
}
struct WriteSpec

View File

@ -5,6 +5,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
ClusterUpdateActions joiningToClusterUpdates(const ClusterConfigPtr & cfg, std::string_view joining)
{
ClusterUpdateActions out;
@ -79,7 +85,7 @@ String serializeClusterConfig(const ClusterConfigPtr & cfg, const ClusterUpdateA
new_config.emplace_back(RaftServerConfig{*cfg->get_server(priority->id)});
}
else
UNREACHABLE();
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected update");
}
for (const auto & item : cfg->get_servers())

View File

@ -990,7 +990,7 @@ KeeperServer::ConfigUpdateState KeeperServer::applyConfigUpdate(
raft_instance->set_priority(update->id, update->priority, /*broadcast on live leader*/true);
return Accepted;
}
UNREACHABLE();
std::unreachable();
}
ClusterUpdateActions KeeperServer::getRaftConfigurationDiff(const Poco::Util::AbstractConfiguration & config)

View File

@ -667,8 +667,6 @@ public:
case Types::AggregateFunctionState: return f(field.template get<AggregateFunctionStateData>());
case Types::CustomType: return f(field.template get<CustomType>());
}
UNREACHABLE();
}
String dump() const;

View File

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

View File

@ -17,7 +17,6 @@ std::string toString(MetadataStorageTransactionState state)
case MetadataStorageTransactionState::PARTIALLY_ROLLED_BACK:
return "PARTIALLY_ROLLED_BACK";
}
UNREACHABLE();
}
}

View File

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

View File

@ -62,7 +62,6 @@ String escapingRuleToString(FormatSettings::EscapingRule escaping_rule)
case FormatSettings::EscapingRule::Raw:
return "Raw";
}
UNREACHABLE();
}
void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)

View File

@ -149,8 +149,6 @@ struct IntegerRoundingComputation
return x;
}
}
UNREACHABLE();
}
static ALWAYS_INLINE T compute(T x, T scale)
@ -163,8 +161,6 @@ struct IntegerRoundingComputation
case ScaleMode::Negative:
return computeImpl(x, scale);
}
UNREACHABLE();
}
static ALWAYS_INLINE void compute(const T * __restrict in, size_t scale, T * __restrict out) requires std::integral<T>
@ -247,8 +243,6 @@ inline float roundWithMode(float x, RoundingMode mode)
case RoundingMode::Ceil: return ceilf(x);
case RoundingMode::Trunc: return truncf(x);
}
UNREACHABLE();
}
inline double roundWithMode(double x, RoundingMode mode)
@ -260,8 +254,6 @@ inline double roundWithMode(double x, RoundingMode mode)
case RoundingMode::Ceil: return ceil(x);
case RoundingMode::Trunc: return trunc(x);
}
UNREACHABLE();
}
template <typename T>

View File

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

View File

@ -381,8 +381,6 @@ bool PointInPolygonWithGrid<CoordinateType>::contains(CoordinateType x, Coordina
case CellType::complexPolygon:
return boost::geometry::within(Point(x, y), polygons[cell.index_of_inner_polygon]);
}
UNREACHABLE();
}

View File

@ -35,7 +35,6 @@ namespace
case UserDefinedSQLObjectType::Function:
return "function_";
}
UNREACHABLE();
}
constexpr std::string_view sql_extension = ".sql";

View File

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

View File

@ -88,7 +88,6 @@ public:
case Status::TOO_LARGE_COMPRESSED_BLOCK:
return "TOO_LARGE_COMPRESSED_BLOCK";
}
UNREACHABLE();
}
explicit HadoopSnappyReadBuffer(

View File

@ -117,8 +117,6 @@ size_t AggregatedDataVariants::size() const
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
size_t AggregatedDataVariants::sizeWithoutOverflowRow() const
@ -136,8 +134,6 @@ size_t AggregatedDataVariants::sizeWithoutOverflowRow() const
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
const char * AggregatedDataVariants::getMethodName() const
@ -155,8 +151,6 @@ const char * AggregatedDataVariants::getMethodName() const
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
bool AggregatedDataVariants::isTwoLevel() const
@ -174,8 +168,6 @@ bool AggregatedDataVariants::isTwoLevel() const
APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
bool AggregatedDataVariants::isConvertibleToTwoLevel() const

View File

@ -799,7 +799,6 @@ String FileSegment::stateToString(FileSegment::State state)
case FileSegment::State::DETACHED:
return "DETACHED";
}
UNREACHABLE();
}
bool FileSegment::assertCorrectness() const

View File

@ -309,7 +309,6 @@ ComparisonGraphCompareResult ComparisonGraph<Node>::pathToCompareResult(Path pat
case Path::GREATER: return inverse ? ComparisonGraphCompareResult::LESS : ComparisonGraphCompareResult::GREATER;
case Path::GREATER_OR_EQUAL: return inverse ? ComparisonGraphCompareResult::LESS_OR_EQUAL : ComparisonGraphCompareResult::GREATER_OR_EQUAL;
}
UNREACHABLE();
}
template <ComparisonGraphNodeType Node>

View File

@ -705,7 +705,6 @@ namespace
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
}
@ -2641,8 +2640,6 @@ private:
default:
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type);
}
UNREACHABLE();
}
template <typename Map>

View File

@ -322,8 +322,6 @@ public:
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
size_t getTotalByteCountImpl(Type which) const
@ -338,8 +336,6 @@ public:
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
size_t getBufferSizeInCells(Type which) const
@ -354,8 +350,6 @@ public:
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
/// NOLINTEND(bugprone-macro-parentheses)
};

View File

@ -33,7 +33,6 @@ BlockIO InterpreterTransactionControlQuery::execute()
case ASTTransactionControl::SET_SNAPSHOT:
return executeSetSnapshot(session_context, tcl.snapshot);
}
UNREACHABLE();
}
BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context)

View File

@ -41,8 +41,6 @@ size_t SetVariantsTemplate<Variant>::getTotalRowCount() const
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
template <typename Variant>
@ -57,8 +55,6 @@ size_t SetVariantsTemplate<Variant>::getTotalByteCount() const
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
UNREACHABLE();
}
template <typename Variant>

View File

@ -40,8 +40,6 @@ public:
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
}
UNREACHABLE();
}
static ExplainKind fromString(const String & str)

View File

@ -42,7 +42,7 @@ Token quotedString(const char *& pos, const char * const token_begin, const char
continue;
}
UNREACHABLE();
chassert(false);
}
}
@ -538,8 +538,6 @@ const char * getTokenName(TokenType type)
APPLY_FOR_TOKENS(M)
#undef M
}
UNREACHABLE();
}

View File

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

View File

@ -36,8 +36,6 @@ std::string IProcessor::statusToName(Status status)
case Status::ExpandPipeline:
return "ExpandPipeline";
}
UNREACHABLE();
}
}

View File

@ -1136,8 +1136,6 @@ static void addMergingFinal(
return std::make_shared<GraphiteRollupSortedTransform>(header, num_outputs,
sort_description, max_block_size_rows, /*max_block_size_bytes=*/0, merging_params.graphite_params, now);
}
UNREACHABLE();
};
pipe.addTransform(get_merging_processor());
@ -2125,8 +2123,6 @@ static const char * indexTypeToString(ReadFromMergeTree::IndexType type)
case ReadFromMergeTree::IndexType::Skip:
return "Skip";
}
UNREACHABLE();
}
static const char * readTypeToString(ReadFromMergeTree::ReadType type)
@ -2142,8 +2138,6 @@ static const char * readTypeToString(ReadFromMergeTree::ReadType type)
case ReadFromMergeTree::ReadType::ParallelReplicas:
return "Parallel";
}
UNREACHABLE();
}
void ReadFromMergeTree::describeActions(FormatSettings & format_settings) const

View File

@ -86,8 +86,6 @@ static String totalsModeToString(TotalsMode totals_mode, double auto_include_thr
case TotalsMode::AFTER_HAVING_AUTO:
return "after_having_auto threshold " + std::to_string(auto_include_threshold);
}
UNREACHABLE();
}
void TotalsHavingStep::describeActions(FormatSettings & settings) const

View File

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

View File

@ -898,8 +898,6 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St
{
return std::current_exception();
}
UNREACHABLE();
}
void FinalizingViewsTransform::work()

View File

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

View File

@ -2964,8 +2964,6 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri
case ALWAYS_TRUE:
return "true";
}
UNREACHABLE();
}

View File

@ -1177,8 +1177,6 @@ String MergeTreeData::MergingParams::getModeName() const
case Graphite: return "Graphite";
case VersionedCollapsing: return "VersionedCollapsing";
}
UNREACHABLE();
}
Int64 MergeTreeData::getMaxBlockNumber() const

View File

@ -360,8 +360,6 @@ Block MergeTreeDataWriter::mergeBlock(
return std::make_shared<GraphiteRollupSortedAlgorithm>(
block, 1, sort_description, block_size + 1, /*block_size_bytes=*/0, merging_params.graphite_params, time(nullptr));
}
UNREACHABLE();
};
auto merging_algorithm = get_merging_algorithm();

View File

@ -616,8 +616,6 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st
}
}
}
UNREACHABLE();
}
void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk)

View File

@ -297,7 +297,6 @@ namespace
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
UNREACHABLE();
}
class AddingAggregatedChunkInfoTransform : public ISimpleTransform
@ -920,7 +919,6 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec)
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
UNREACHABLE();
}
UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec)
@ -948,7 +946,6 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec)
CASE_WINDOW_KIND(Year)
#undef CASE_WINDOW_KIND
}
UNREACHABLE();
}
void StorageWindowView::addFireSignal(std::set<UInt32> & signals)