mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #64435 from rschu1ze/revert-revert-64035-unreachable-unreachable
Revert "Revert "Remove some unnecessary UNREACHABLEs""
This commit is contained in:
commit
d776d88797
@ -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);
|
||||
|
@ -155,8 +155,8 @@ auto instructionFailToString(InstructionFail fail)
|
||||
ret("AVX2");
|
||||
case InstructionFail::AVX512:
|
||||
ret("AVX512");
|
||||
#undef ret
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -258,7 +258,7 @@ namespace
|
||||
case TABLE_LEVEL: return AccessFlags::allFlagsGrantableOnTableLevel();
|
||||
case COLUMN_LEVEL: return AccessFlags::allFlagsGrantableOnColumnLevel();
|
||||
}
|
||||
UNREACHABLE();
|
||||
chassert(false);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -463,7 +463,6 @@ public:
|
||||
return "sumWithOverflow";
|
||||
else if constexpr (Type == AggregateFunctionTypeSumKahan)
|
||||
return "sumKahan";
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
explicit AggregateFunctionSum(const DataTypes & argument_types_)
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -54,8 +54,6 @@ String toString(TargetArch arch)
|
||||
case TargetArch::AMXTILE: return "amxtile";
|
||||
case TargetArch::AMXINT8: return "amxint8";
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -75,7 +75,6 @@ const char * TasksStatsCounters::metricsProviderString(MetricsProvider provider)
|
||||
case MetricsProvider::Netlink:
|
||||
return "netlink";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
bool TasksStatsCounters::checkIfAvailable()
|
||||
|
@ -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)
|
||||
|
@ -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()
|
||||
|
@ -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
|
||||
|
@ -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())
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -36,7 +36,6 @@ String ISerialization::kindToString(Kind kind)
|
||||
case Kind::SPARSE:
|
||||
return "Sparse";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
ISerialization::Kind ISerialization::stringToKind(const String & str)
|
||||
|
@ -17,7 +17,6 @@ std::string toString(MetadataStorageTransactionState state)
|
||||
case MetadataStorageTransactionState::PARTIALLY_ROLLED_BACK:
|
||||
return "PARTIALLY_ROLLED_BACK";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -35,7 +35,6 @@ namespace
|
||||
case UserDefinedSQLObjectType::Function:
|
||||
return "function_";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
constexpr std::string_view sql_extension = ".sql";
|
||||
|
@ -52,7 +52,6 @@ std::string toContentEncodingName(CompressionMethod method)
|
||||
case CompressionMethod::None:
|
||||
return "";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
CompressionMethod chooseHTTPCompressionMethod(const std::string & list)
|
||||
|
@ -88,7 +88,6 @@ public:
|
||||
case Status::TOO_LARGE_COMPRESSED_BLOCK:
|
||||
return "TOO_LARGE_COMPRESSED_BLOCK";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
explicit HadoopSnappyReadBuffer(
|
||||
|
@ -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
|
||||
|
@ -799,7 +799,6 @@ String FileSegment::stateToString(FileSegment::State state)
|
||||
case FileSegment::State::DETACHED:
|
||||
return "DETACHED";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
bool FileSegment::assertCorrectness() const
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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)
|
||||
};
|
||||
|
@ -33,7 +33,6 @@ BlockIO InterpreterTransactionControlQuery::execute()
|
||||
case ASTTransactionControl::SET_SNAPSHOT:
|
||||
return executeSetSnapshot(session_context, tcl.snapshot);
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
BlockIO InterpreterTransactionControlQuery::executeBegin(ContextMutablePtr session_context)
|
||||
|
@ -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>
|
||||
|
@ -40,8 +40,6 @@ public:
|
||||
case TableOverride: return "EXPLAIN TABLE OVERRIDE";
|
||||
case CurrentTransaction: return "EXPLAIN CURRENT TRANSACTION";
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
static ExplainKind fromString(const String & str)
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
@ -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()
|
||||
|
@ -36,8 +36,6 @@ std::string IProcessor::statusToName(Status status)
|
||||
case Status::ExpandPipeline:
|
||||
return "ExpandPipeline";
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
@ -898,8 +898,6 @@ static std::exception_ptr addStorageToException(std::exception_ptr ptr, const St
|
||||
{
|
||||
return std::current_exception();
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
void FinalizingViewsTransform::work()
|
||||
|
@ -93,7 +93,6 @@ String BackgroundJobsAssignee::toString(Type type)
|
||||
case Type::Moving:
|
||||
return "Moving";
|
||||
}
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
void BackgroundJobsAssignee::start()
|
||||
|
@ -2964,8 +2964,6 @@ String KeyCondition::RPNElement::toString(std::string_view column_name, bool pri
|
||||
case ALWAYS_TRUE:
|
||||
return "true";
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
|
||||
|
@ -1177,8 +1177,6 @@ String MergeTreeData::MergingParams::getModeName() const
|
||||
case Graphite: return "Graphite";
|
||||
case VersionedCollapsing: return "VersionedCollapsing";
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
Int64 MergeTreeData::getMaxBlockNumber() const
|
||||
|
@ -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();
|
||||
|
@ -616,8 +616,6 @@ PartMovesBetweenShardsOrchestrator::Entry PartMovesBetweenShardsOrchestrator::st
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
UNREACHABLE();
|
||||
}
|
||||
|
||||
void PartMovesBetweenShardsOrchestrator::removePins(const Entry & entry, zkutil::ZooKeeperPtr zk)
|
||||
|
@ -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)
|
||||
|
Loading…
Reference in New Issue
Block a user