Initial: replacing hardcoded toString for enums with magic_enum

This commit is contained in:
Mike Kot 2021-09-06 16:24:03 +02:00
parent b40b4b0b4d
commit 8e9aacadd1
50 changed files with 198 additions and 463 deletions

3
.gitmodules vendored
View File

@ -246,3 +246,6 @@
[submodule "contrib/bzip2"] [submodule "contrib/bzip2"]
path = contrib/bzip2 path = contrib/bzip2
url = https://github.com/ClickHouse-Extras/bzip2.git url = https://github.com/ClickHouse-Extras/bzip2.git
[submodule "contrib/magic_enum"]
path = contrib/magic_enum
url = https://github.com/Neargye/magic_enum

View File

@ -0,0 +1,32 @@
#pragma once
#include <type_traits>
#include <fmt/format.h>
#include <../../contrib/magic_enum/include/magic_enum.hpp> //FIXME
// Can't name concept is_enum as is would conflict with type trait
template <class T> concept type_is_enum = std::is_enum_v<T>;
// Enable printing enum values as strings via fmt + magic_enum
template <type_is_enum T>
struct fmt::formatter<T> : fmt::formatter<std::string_view> {
constexpr auto format(T enum_value, auto& format_context) {
return formatter<string_view>::format(
magic_enum::enum_name(enum_value), format_context);
}
};
namespace DB
{
/**
* Some enums have multiple-word values like FLUSH_DICT. However, we should parse user input like FLUSH DICT.
* magic_enum::enum_names returns names with underscore, so we have to replace underscore with space.
*/
std::string UnderscoreToSpace(std::string_view str)
{
std::string out(str.data(), str.size());
std::replace(out.begin(), out.end(), '_', ' ');
return out;
}
}

View File

@ -32,6 +32,8 @@ endif()
set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1) set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1)
include_directories("${CMAKE_SOURCE_DIR}/contrib/magic_enum/include")
add_subdirectory (abseil-cpp-cmake) add_subdirectory (abseil-cpp-cmake)
add_subdirectory (boost-cmake) add_subdirectory (boost-cmake)
add_subdirectory (cctz-cmake) add_subdirectory (cctz-cmake)

1
contrib/magic_enum vendored Submodule

@ -0,0 +1 @@
Subproject commit 38f86e4d093cfc9034a140d37de2168e3951bef3

View File

@ -506,8 +506,9 @@ static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Ar
void ColumnAggregateFunction::insert(const Field & x) void ColumnAggregateFunction::insert(const Field & x)
{ {
if (x.getType() != Field::Types::AggregateFunctionState) if (x.getType() != Field::Types::AggregateFunctionState)
throw Exception(String("Inserting field of type ") + x.getTypeName() + " into ColumnAggregateFunction. " throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected " + Field::Types::toString(Field::Types::AggregateFunctionState), ErrorCodes::LOGICAL_ERROR); "Inserting field of type {} into ColumnAggregateFunction. Expected {}",
x.getTypeName(), Field::Types::AggregateFunctionState);
const auto & field_name = x.get<const AggregateFunctionStateData &>().name; const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
if (type_string != field_name) if (type_string != field_name)

View File

@ -1,39 +0,0 @@
#include <Common/ExternalLoaderStatus.h>
namespace DB
{
String toString(ExternalLoaderStatus status)
{
using Status = ExternalLoaderStatus;
switch (status)
{
case Status::NOT_LOADED: return "NOT_LOADED";
case Status::LOADED: return "LOADED";
case Status::FAILED: return "FAILED";
case Status::LOADING: return "LOADING";
case Status::FAILED_AND_RELOADING: return "FAILED_AND_RELOADING";
case Status::LOADED_AND_RELOADING: return "LOADED_AND_RELOADING";
case Status::NOT_EXIST: return "NOT_EXIST";
}
__builtin_unreachable();
}
std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues()
{
using Status = ExternalLoaderStatus;
return std::vector<std::pair<String, Int8>>{
{toString(Status::NOT_LOADED), static_cast<Int8>(Status::NOT_LOADED)},
{toString(Status::LOADED), static_cast<Int8>(Status::LOADED)},
{toString(Status::FAILED), static_cast<Int8>(Status::FAILED)},
{toString(Status::LOADING), static_cast<Int8>(Status::LOADING)},
{toString(Status::LOADED_AND_RELOADING), static_cast<Int8>(Status::LOADED_AND_RELOADING)},
{toString(Status::FAILED_AND_RELOADING), static_cast<Int8>(Status::FAILED_AND_RELOADING)},
{toString(Status::NOT_EXIST), static_cast<Int8>(Status::NOT_EXIST)},
};
}
std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status)
{
return out << toString(status);
}
}

View File

@ -4,10 +4,11 @@
#include <utility> #include <utility>
#include <ostream> #include <ostream>
#include <common/types.h> #include <common/types.h>
#include <common/EnumReflection.h>
namespace DB namespace DB
{ {
enum class ExternalLoaderStatus enum class ExternalLoaderStatus : Int8
{ {
NOT_LOADED, /// Object hasn't been tried to load. This is an initial state. NOT_LOADED, /// Object hasn't been tried to load. This is an initial state.
LOADED, /// Object has been loaded successfully. LOADED, /// Object has been loaded successfully.
@ -18,7 +19,14 @@ namespace DB
NOT_EXIST, /// Object with this name wasn't found in the configuration. NOT_EXIST, /// Object with this name wasn't found in the configuration.
}; };
String toString(ExternalLoaderStatus status); std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues()
std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues(); {
std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status); std::vector<std::pair<String, Int8>> out;
out.reserve(magic_enum::enum_count<ExternalLoaderStatus>());
for (const auto & [value, str] : magic_enum::enum_entries<ExternalLoaderStatus>())
out.emplace_back(str, value);
return out;
}
} }

View File

@ -9,23 +9,6 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR; extern const int SYNTAX_ERROR;
} }
const char * IntervalKind::toString() const
{
switch (kind)
{
case IntervalKind::Second: return "Second";
case IntervalKind::Minute: return "Minute";
case IntervalKind::Hour: return "Hour";
case IntervalKind::Day: return "Day";
case IntervalKind::Week: return "Week";
case IntervalKind::Month: return "Month";
case IntervalKind::Quarter: return "Quarter";
case IntervalKind::Year: return "Year";
}
__builtin_unreachable();
}
Int32 IntervalKind::toAvgSeconds() const Int32 IntervalKind::toAvgSeconds() const
{ {
switch (kind) switch (kind)

View File

@ -1,7 +1,7 @@
#pragma once #pragma once
#include <common/types.h> #include <common/types.h>
#include <common/EnumReflection.h>
namespace DB namespace DB
{ {
@ -24,7 +24,7 @@ struct IntervalKind
IntervalKind(Kind kind_ = Second) : kind(kind_) {} IntervalKind(Kind kind_ = Second) : kind(kind_) {}
operator Kind() const { return kind; } operator Kind() const { return kind; }
const char * toString() const; constexpr std::string_view toString() const { return magic_enum::enum_name(kind); }
/// Returns number of seconds in one interval. /// Returns number of seconds in one interval.
/// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds.

View File

@ -33,7 +33,7 @@ using ItemPtr = std::unique_ptr<IItem>;
class JSONString : public IItem class JSONString : public IItem
{ {
public: public:
explicit JSONString(std::string value_) : value(std::move(value_)) {} JSONString(std::string_view value_) : value(value_) {}
void format(const FormatSettings & settings, FormatContext & context) override; void format(const FormatSettings & settings, FormatContext & context) override;
private: private:
@ -97,6 +97,7 @@ public:
void add(std::string key, ItemPtr value) { values.emplace_back(Pair{.key = std::move(key), .value = std::move(value)}); } void add(std::string key, ItemPtr value) { values.emplace_back(Pair{.key = std::move(key), .value = std::move(value)}); }
void add(std::string key, std::string value) { add(std::move(key), std::make_unique<JSONString>(std::move(value))); } void add(std::string key, std::string value) { add(std::move(key), std::make_unique<JSONString>(std::move(value))); }
void add(std::string key, const char * value) { add(std::move(key), std::make_unique<JSONString>(value)); } void add(std::string key, const char * value) { add(std::move(key), std::make_unique<JSONString>(value)); }
void add(std::string key, std::string_view value) { add(std::move(key), std::make_unique<JSONString>(value)); }
void add(std::string key, bool value) { add(std::move(key), std::make_unique<JSONBool>(std::move(value))); } void add(std::string key, bool value) { add(std::move(key), std::make_unique<JSONBool>(std::move(value))); }
template <typename T, std::enable_if_t<std::is_arithmetic<T>::value, bool> = true> template <typename T, std::enable_if_t<std::is_arithmetic<T>::value, bool> = true>

View File

@ -15,7 +15,6 @@
#include <common/DayNum.h> #include <common/DayNum.h>
#include <common/strong_typedef.h> #include <common/strong_typedef.h>
namespace DB namespace DB
{ {
@ -283,33 +282,6 @@ public:
Map = 26, Map = 26,
UUID = 27, UUID = 27,
}; };
static const char * toString(Which which)
{
switch (which)
{
case Null: return "Null";
case UInt64: return "UInt64";
case UInt128: return "UInt128";
case UInt256: return "UInt256";
case Int64: return "Int64";
case Int128: return "Int128";
case Int256: return "Int256";
case UUID: return "UUID";
case Float64: return "Float64";
case String: return "String";
case Array: return "Array";
case Tuple: return "Tuple";
case Map: return "Map";
case Decimal32: return "Decimal32";
case Decimal64: return "Decimal64";
case Decimal128: return "Decimal128";
case Decimal256: return "Decimal256";
case AggregateFunctionState: return "AggregateFunctionState";
}
throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD);
}
}; };
@ -417,6 +389,8 @@ public:
Types::Which getType() const { return which; } Types::Which getType() const { return which; }
//static constexpr std::string_view getTypeName() { return Types::toString(which); }
const char * getTypeName() const { return Types::toString(which); } const char * getTypeName() const { return Types::toString(which); }
bool isNull() const { return which == Types::Null; } bool isNull() const { return which == Types::Null; }
@ -808,7 +782,8 @@ NearestFieldType<std::decay_t<T>> & Field::get()
constexpr Field::Types::Which target = TypeToEnum<StoredType>::value; constexpr Field::Types::Which target = TypeToEnum<StoredType>::value;
if (target != which if (target != which
&& (!isInt64OrUInt64FieldType(target) || !isInt64OrUInt64FieldType(which))) && (!isInt64OrUInt64FieldType(target) || !isInt64OrUInt64FieldType(which)))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid Field get from type {} to type {}", Types::toString(which), Types::toString(target)); throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid Field get from type {} to type {}", which, target);
#endif #endif
StoredType * MAY_ALIAS ptr = reinterpret_cast<StoredType *>(&storage); StoredType * MAY_ALIAS ptr = reinterpret_cast<StoredType *>(&storage);
@ -821,8 +796,11 @@ template <typename T>
auto & Field::safeGet() auto & Field::safeGet()
{ {
const Types::Which requested = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value; const Types::Which requested = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
if (which != requested) if (which != requested)
throw Exception("Bad get: has " + std::string(getTypeName()) + ", requested " + std::string(Types::toString(requested)), ErrorCodes::BAD_GET); throw Exception(ErrorCodes::BAD_GET,
"Bad get: has {}, requested {}", getTypeName(), requested);
return get<T>(); return get<T>();
} }

View File

@ -27,7 +27,7 @@ public:
DataTypeInterval(IntervalKind kind_) : kind(kind_) {} DataTypeInterval(IntervalKind kind_) : kind(kind_) {}
std::string doGetName() const override { return std::string("Interval") + kind.toString(); } std::string doGetName() const override { return fmt::format("Interval{}", kind.toString()); }
const char * getFamilyName() const override { return "Interval"; } const char * getFamilyName() const override { return "Interval"; }
TypeIndex getTypeId() const override { return TypeIndex::Interval; } TypeIndex getTypeId() const override { return TypeIndex::Interval; }

View File

@ -26,10 +26,8 @@ String getExceptionMessage(
const String & message, size_t argument_index, const char * argument_name, const String & message, size_t argument_index, const char * argument_name,
const std::string & context_data_type_name, Field::Types::Which field_type) const std::string & context_data_type_name, Field::Types::Which field_type)
{ {
return std::string("Parameter #") + std::to_string(argument_index) + " '" return fmt::format("Parameter #{} '{}' for {}{}, expected {} literal",
+ argument_name + "' for " + context_data_type_name argument_index, argument_name, context_data_type_name, message, field_type);
+ message
+ ", expected: " + Field::Types::toString(field_type) + " literal.";
} }
template <typename T, ArgumentKind Kind> template <typename T, ArgumentKind Kind>

View File

@ -15,22 +15,6 @@ namespace ErrorCodes
extern const int UNKNOWN_VOLUME_TYPE; extern const int UNKNOWN_VOLUME_TYPE;
} }
String volumeTypeToString(VolumeType type)
{
switch (type)
{
case VolumeType::JBOD:
return "JBOD";
case VolumeType::RAID1:
return "RAID1";
case VolumeType::SINGLE_DISK:
return "SINGLE_DISK";
case VolumeType::UNKNOWN:
return "UNKNOWN";
}
throw Exception("Unknown volume type, please add it to DB::volumeTypeToString", ErrorCodes::UNKNOWN_VOLUME_TYPE);
}
IVolume::IVolume( IVolume::IVolume(
String name_, String name_,
const Poco::Util::AbstractConfiguration & config, const Poco::Util::AbstractConfiguration & config,

View File

@ -16,8 +16,6 @@ enum class VolumeType
UNKNOWN UNKNOWN
}; };
String volumeTypeToString(VolumeType t);
class IVolume; class IVolume;
using VolumePtr = std::shared_ptr<IVolume>; using VolumePtr = std::shared_ptr<IVolume>;
using Volumes = std::vector<VolumePtr>; using Volumes = std::vector<VolumePtr>;

View File

@ -632,7 +632,9 @@ class FunctionBinaryArithmetic : public IFunction
std::string function_name; std::string function_name;
if (interval_data_type) if (interval_data_type)
{ {
function_name = String(is_plus ? "add" : "subtract") + interval_data_type->getKind().toString() + 's'; function_name = fmt::format("{}{}s",
is_plus ? "add" : "subtract",
interval_data_type->getKind().toString());
} }
else else
{ {

View File

@ -261,10 +261,10 @@ public:
+ ". This argument is optional and must be a constant string with timezone name", + ". This argument is optional and must be a constant string with timezone name",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (first_argument_is_date && result_type_is_date) if (first_argument_is_date && result_type_is_date)
throw Exception( throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function " + getName() + " with interval type " + interval_type->getKind().toString() "The timezone argument of function {} with interval type {} is allowed only when the 1st argument "
+ " is allowed only when the 1st argument has the type DateTime", "has the type DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); getName(), interval_type->getKind().toString());
}; };
if (arguments.size() == 2) if (arguments.size() == 2)

View File

@ -6,6 +6,9 @@
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <functional> #include <functional>
#include <type_traits>
#include <common/EnumReflection.h>
namespace DB namespace DB
@ -41,16 +44,47 @@ struct QuoteManipReadBuffer : std::reference_wrapper<ReadBuffer> { usin
struct DoubleQuoteManipReadBuffer : std::reference_wrapper<ReadBuffer> { using std::reference_wrapper<ReadBuffer>::reference_wrapper; }; struct DoubleQuoteManipReadBuffer : std::reference_wrapper<ReadBuffer> { using std::reference_wrapper<ReadBuffer>::reference_wrapper; };
struct BinaryManipReadBuffer : std::reference_wrapper<ReadBuffer> { using std::reference_wrapper<ReadBuffer>::reference_wrapper; }; struct BinaryManipReadBuffer : std::reference_wrapper<ReadBuffer> { using std::reference_wrapper<ReadBuffer>::reference_wrapper; };
inline WriteBuffer & operator<<(WriteBuffer & buf, const auto & x)
{
writeText(x, buf);
return buf;
}
template <typename T> WriteBuffer & operator<< (WriteBuffer & buf, const T & x) { writeText(x, buf); return buf; } inline WriteBuffer & operator<<(WriteBuffer & buf, auto x) requires(std::is_enum_v<decltype(x)>)
/// If you do not use the manipulators, the string is displayed without an escape, as is. {
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const String & x) { writeString(x, buf); return buf; } writeString(magic_enum::enum_name(x), buf);
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const std::string_view & x) { writeString(StringRef(x), buf); return buf; } return buf;
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const StringRef & x) { writeString(x, buf); return buf; } }
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const char & x) { writeChar(x, buf); return buf; }
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const pcg32_fast & x) { PcgSerializer::serializePcg32(x, buf); return buf; }
inline WriteBuffer & operator<< (WriteBuffer & buf, const char * x) { writeCString(x, buf); return buf; } inline WriteBuffer & operator<<(WriteBuffer & buf, std::string_view x)
{
writeString(x, buf);
return buf;
}
inline WriteBuffer & operator<<(WriteBuffer & buf, StringRef x)
{
writeString(x, buf);
return buf;
}
inline WriteBuffer & operator<<(WriteBuffer & buf, char x)
{
writeChar(x, buf);
return buf;
}
inline WriteBuffer & operator<<(WriteBuffer & buf, const char * x)
{
writeCString(x, buf);
return buf;
}
WriteBuffer & operator<<(WriteBuffer & buf, const pcg32_fast & x)
{
PcgSerializer::serializePcg32(x, buf);
return buf;
}
inline EscapeManipWriteBuffer operator<< (WriteBuffer & buf, EscapeManip) { return buf; } inline EscapeManipWriteBuffer operator<< (WriteBuffer & buf, EscapeManip) { return buf; }
inline QuoteManipWriteBuffer operator<< (WriteBuffer & buf, QuoteManip) { return buf; } inline QuoteManipWriteBuffer operator<< (WriteBuffer & buf, QuoteManip) { return buf; }

View File

@ -42,7 +42,6 @@
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
namespace DB namespace DB
{ {
@ -177,17 +176,20 @@ inline void writeFloatText(T x, WriteBuffer & buf)
buf.write(buffer, result); buf.write(buffer, result);
} }
inline void writeString(const char * data, size_t size, WriteBuffer & buf) inline void writeString(const char * data, size_t size, WriteBuffer & buf)
{ {
buf.write(data, size); buf.write(data, size);
} }
inline void writeString(const StringRef & ref, WriteBuffer & buf) inline void writeString(StringRef ref, WriteBuffer & buf)
{ {
writeString(ref.data, ref.size, buf); writeString(ref.data, ref.size, buf);
} }
inline void writeString(std::string_view ref, WriteBuffer & buf)
{
writeString(ref.data(), ref.size(), buf);
}
/** Writes a C-string without creating a temporary object. If the string is a literal, then `strlen` is executed at the compilation stage. /** Writes a C-string without creating a temporary object. If the string is a literal, then `strlen` is executed at the compilation stage.
* Use when the string is a literal. * Use when the string is a literal.

View File

@ -30,34 +30,15 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
const char * ActionsDAG::typeToString(ActionsDAG::ActionType type)
{
switch (type)
{
case ActionType::INPUT:
return "Input";
case ActionType::COLUMN:
return "Column";
case ActionType::ALIAS:
return "Alias";
case ActionType::ARRAY_JOIN:
return "ArrayJoin";
case ActionType::FUNCTION:
return "Function";
}
__builtin_unreachable();
}
void ActionsDAG::Node::toTree(JSONBuilder::JSONMap & map) const void ActionsDAG::Node::toTree(JSONBuilder::JSONMap & map) const
{ {
map.add("Node Type", ActionsDAG::typeToString(type)); map.add("Node Type", magic_enum::enum_name(type));
if (result_type) if (result_type)
map.add("Result Type", result_type->getName()); map.add("Result Type", result_type->getName());
if (!result_name.empty()) if (!result_name.empty())
map.add("Result Type", ActionsDAG::typeToString(type)); map.add("Result Type", magic_enum::enum_name(type));
if (column) if (column)
map.add("Column", column->getName()); map.add("Column", column->getName());

View File

@ -61,8 +61,6 @@ public:
FUNCTION, FUNCTION,
}; };
static const char * typeToString(ActionType type);
struct Node; struct Node;
using NodeRawPtrs = std::vector<Node *>; using NodeRawPtrs = std::vector<Node *>;
using NodeRawConstPtrs = std::vector<const Node *>; using NodeRawConstPtrs = std::vector<const Node *>;

View File

@ -624,7 +624,7 @@ void makeWindowDescriptionFromAST(const Context & context,
{ {
throw Exception(ErrorCodes::NOT_IMPLEMENTED, throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Window frame '{}' is not implemented (while processing '{}')", "Window frame '{}' is not implemented (while processing '{}')",
WindowFrame::toString(definition.frame_type), definition.frame_type,
ast->formatForErrorMessage()); ast->formatForErrorMessage());
} }

View File

@ -426,7 +426,7 @@ BlockIO InterpreterSystemQuery::execute()
} }
case Type::STOP_LISTEN_QUERIES: case Type::STOP_LISTEN_QUERIES:
case Type::START_LISTEN_QUERIES: case Type::START_LISTEN_QUERIES:
throw Exception(String(ASTSystemQuery::typeToString(query.type)) + " is not supported yet", ErrorCodes::NOT_IMPLEMENTED); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "{} is not supported yet", query.type);
default: default:
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS); throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
} }

View File

@ -51,7 +51,7 @@ std::string WindowFrame::toString() const
void WindowFrame::toString(WriteBuffer & buf) const void WindowFrame::toString(WriteBuffer & buf) const
{ {
buf << toString(type) << " BETWEEN "; buf << type << " BETWEEN ";
if (begin_type == BoundaryType::Current) if (begin_type == BoundaryType::Current)
{ {
buf << "CURRENT ROW"; buf << "CURRENT ROW";
@ -101,9 +101,9 @@ void WindowFrame::checkValid() const
{ {
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", "Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
toString(type), type,
applyVisitor(FieldVisitorToString(), begin_offset), applyVisitor(FieldVisitorToString(), begin_offset),
Field::Types::toString(begin_offset.getType())); begin_offset.getType());
} }
if (end_type == BoundaryType::Offset if (end_type == BoundaryType::Offset
@ -114,9 +114,9 @@ void WindowFrame::checkValid() const
{ {
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given", "Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
toString(type), type,
applyVisitor(FieldVisitorToString(), end_offset), applyVisitor(FieldVisitorToString(), end_offset),
Field::Types::toString(end_offset.getType())); end_offset.getType());
} }
} }

View File

@ -73,40 +73,6 @@ struct WindowFrame
&& other.end_preceding == end_preceding && other.end_preceding == end_preceding
; ;
} }
static std::string toString(FrameType type)
{
switch (type)
{
case FrameType::Rows:
return "ROWS";
case FrameType::Groups:
return "GROUPS";
case FrameType::Range:
return "RANGE";
}
// Somehow GCC 10 doesn't understand that the above switch is exhaustive.
assert(false);
return "<unknown frame>";
}
static std::string toString(BoundaryType type)
{
switch (type)
{
case BoundaryType::Unbounded:
return "UNBOUNDED";
case BoundaryType::Offset:
return "OFFSET";
case BoundaryType::Current:
return "CURRENT ROW";
}
// Somehow GCC 10 doesn't understand that the above switch is exhaustive.
assert(false);
return "<unknown frame boundary>";
}
}; };
struct WindowDescription struct WindowDescription

View File

@ -75,8 +75,8 @@ static Field convertNumericType(const Field & from, const IDataType & type)
if (from.getType() == Field::Types::Int256) if (from.getType() == Field::Types::Int256)
return convertNumericTypeImpl<Int256, To>(from); return convertNumericTypeImpl<Int256, To>(from);
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
+ Field::Types::toString(from.getType()), ErrorCodes::TYPE_MISMATCH); type.getName(), from.getType());
} }
@ -134,8 +134,8 @@ static Field convertDecimalType(const Field & from, const To & type)
if (from.getType() == Field::Types::Decimal128) if (from.getType() == Field::Types::Decimal128)
return convertDecimalToDecimalType<Decimal128>(from, type); return convertDecimalToDecimalType<Decimal128>(from, type);
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
+ Field::Types::toString(from.getType()), ErrorCodes::TYPE_MISMATCH); type.getName(), from.getType());
} }
@ -387,8 +387,8 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
return convertFieldToType(parsed, type, from_type_hint); return convertFieldToType(parsed, type, from_type_hint);
} }
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: " throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
+ Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH); type.getName(), src.getType());
} }
} }

View File

@ -13,101 +13,10 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
const char * ASTSystemQuery::typeToString(Type type)
{
switch (type)
{
case Type::SHUTDOWN:
return "SHUTDOWN";
case Type::KILL:
return "KILL";
case Type::SUSPEND:
return "SUSPEND";
case Type::DROP_DNS_CACHE:
return "DROP DNS CACHE";
case Type::DROP_MARK_CACHE:
return "DROP MARK CACHE";
case Type::DROP_UNCOMPRESSED_CACHE:
return "DROP UNCOMPRESSED CACHE";
case Type::DROP_MMAP_CACHE:
return "DROP MMAP CACHE";
#if USE_EMBEDDED_COMPILER
case Type::DROP_COMPILED_EXPRESSION_CACHE:
return "DROP COMPILED EXPRESSION CACHE";
#endif
case Type::STOP_LISTEN_QUERIES:
return "STOP LISTEN QUERIES";
case Type::START_LISTEN_QUERIES:
return "START LISTEN QUERIES";
case Type::RESTART_REPLICAS:
return "RESTART REPLICAS";
case Type::RESTART_REPLICA:
return "RESTART REPLICA";
case Type::RESTORE_REPLICA:
return "RESTORE REPLICA";
case Type::DROP_REPLICA:
return "DROP REPLICA";
case Type::SYNC_REPLICA:
return "SYNC REPLICA";
case Type::FLUSH_DISTRIBUTED:
return "FLUSH DISTRIBUTED";
case Type::RELOAD_DICTIONARY:
return "RELOAD DICTIONARY";
case Type::RELOAD_DICTIONARIES:
return "RELOAD DICTIONARIES";
case Type::RELOAD_MODEL:
return "RELOAD MODEL";
case Type::RELOAD_MODELS:
return "RELOAD MODELS";
case Type::RELOAD_EMBEDDED_DICTIONARIES:
return "RELOAD EMBEDDED DICTIONARIES";
case Type::RELOAD_CONFIG:
return "RELOAD CONFIG";
case Type::RELOAD_SYMBOLS:
return "RELOAD SYMBOLS";
case Type::STOP_MERGES:
return "STOP MERGES";
case Type::START_MERGES:
return "START MERGES";
case Type::STOP_TTL_MERGES:
return "STOP TTL MERGES";
case Type::START_TTL_MERGES:
return "START TTL MERGES";
case Type::STOP_MOVES:
return "STOP MOVES";
case Type::START_MOVES:
return "START MOVES";
case Type::STOP_FETCHES:
return "STOP FETCHES";
case Type::START_FETCHES:
return "START FETCHES";
case Type::STOP_REPLICATED_SENDS:
return "STOP REPLICATED SENDS";
case Type::START_REPLICATED_SENDS:
return "START REPLICATED SENDS";
case Type::STOP_REPLICATION_QUEUES:
return "STOP REPLICATION QUEUES";
case Type::START_REPLICATION_QUEUES:
return "START REPLICATION QUEUES";
case Type::STOP_DISTRIBUTED_SENDS:
return "STOP DISTRIBUTED SENDS";
case Type::START_DISTRIBUTED_SENDS:
return "START DISTRIBUTED SENDS";
case Type::FLUSH_LOGS:
return "FLUSH LOGS";
case Type::RESTART_DISK:
return "RESTART DISK";
default:
throw Exception("Unknown SYSTEM query command", ErrorCodes::LOGICAL_ERROR);
}
}
void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{ {
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM "; settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM ";
settings.ostr << typeToString(type) << (settings.hilite ? hilite_none : ""); settings.ostr << type << (settings.hilite ? hilite_none : "");
auto print_database_table = [&] auto print_database_table = [&]
{ {

View File

@ -62,8 +62,6 @@ public:
END END
}; };
static const char * typeToString(Type type);
Type type = Type::UNKNOWN; Type type = Type::UNKNOWN;
String target_model; String target_model;

View File

@ -98,7 +98,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings,
settings.ostr << " "; settings.ostr << " ";
} }
settings.ostr << WindowFrame::toString(frame_type) << " BETWEEN "; settings.ostr << frame_type << " BETWEEN ";
if (frame_begin_type == WindowFrame::BoundaryType::Current) if (frame_begin_type == WindowFrame::BoundaryType::Current)
{ {
settings.ostr << "CURRENT ROW"; settings.ostr << "CURRENT ROW";

View File

@ -14,31 +14,18 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
} }
ParserKeyword::ParserKeyword(const char * s_) : s(s_)
{
}
const char * ParserKeyword::getName() const
{
return s;
}
bool ParserKeyword::parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) bool ParserKeyword::parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected)
{ {
if (pos->type != TokenType::BareWord) if (pos->type != TokenType::BareWord)
return false; return false;
const char * current_word = s; const char * current_word = s.begin();
const char * s_end = s.end();
const size_t s_length = s.length();
size_t s_length = strlen(s);
if (!s_length) if (!s_length)
throw Exception("Logical error: keyword cannot be empty string", ErrorCodes::LOGICAL_ERROR); throw Exception("Logical error: keyword cannot be empty string", ErrorCodes::LOGICAL_ERROR);
const char * s_end = s + s_length;
while (true) while (true)
{ {
expected.add(pos, current_word); expected.add(pos, current_word);

View File

@ -13,13 +13,13 @@ namespace DB
class ParserKeyword : public IParserBase class ParserKeyword : public IParserBase
{ {
private: private:
const char * s; std::string_view s;
public: public:
ParserKeyword(const char * s_); constexpr ParserKeyword(std::string_view s_): s(s_) {}
protected: protected:
const char * getName() const override; const char * getName() const override { return s.data(); }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };

View File

@ -6,6 +6,8 @@
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <Parsers/parseDatabaseAndTableName.h> #include <Parsers/parseDatabaseAndTableName.h>
#include <span>
#include <common/EnumReflection.h>
namespace ErrorCodes namespace ErrorCodes
{ {
@ -67,13 +69,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
auto res = std::make_shared<ASTSystemQuery>(); auto res = std::make_shared<ASTSystemQuery>();
bool found = false; bool found = false;
for (int i = static_cast<int>(Type::UNKNOWN) + 1; i < static_cast<int>(Type::END); ++i)
for (const auto & [entry, str] : magic_enum::enum_entries<Type>())
{ {
Type t = static_cast<Type>(i); if (ParserKeyword(UnderscoreToSpace(str)).ignore(pos, expected))
if (ParserKeyword{ASTSystemQuery::typeToString(t)}.ignore(pos, expected))
{ {
res->type = t; res->type = entry;
found = true; found = true;
break;
} }
} }

View File

@ -617,8 +617,8 @@ void WindowTransform::advanceFrameStart()
default: default:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Frame start type '{}' for frame '{}' is not implemented", "Frame start type '{}' for frame '{}' is not implemented",
WindowFrame::toString(window_description.frame.begin_type), window_description.frame.begin_type,
WindowFrame::toString(window_description.frame.type)); window_description.frame.type);
} }
break; break;
} }
@ -849,7 +849,7 @@ void WindowTransform::advanceFrameEnd()
default: default:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"The frame end type '{}' is not implemented", "The frame end type '{}' is not implemented",
WindowFrame::toString(window_description.frame.end_type)); window_description.frame.end_type);
} }
break; break;
} }

View File

@ -842,53 +842,6 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
return result; return result;
} }
String alterTypeToString(const AlterCommand::Type type)
{
switch (type)
{
case AlterCommand::Type::ADD_COLUMN:
return "ADD COLUMN";
case AlterCommand::Type::ADD_CONSTRAINT:
return "ADD CONSTRAINT";
case AlterCommand::Type::ADD_INDEX:
return "ADD INDEX";
case AlterCommand::Type::ADD_PROJECTION:
return "ADD PROJECTION";
case AlterCommand::Type::COMMENT_COLUMN:
return "COMMENT COLUMN";
case AlterCommand::Type::DROP_COLUMN:
return "DROP COLUMN";
case AlterCommand::Type::DROP_CONSTRAINT:
return "DROP CONSTRAINT";
case AlterCommand::Type::DROP_INDEX:
return "DROP INDEX";
case AlterCommand::Type::DROP_PROJECTION:
return "DROP PROJECTION";
case AlterCommand::Type::MODIFY_COLUMN:
return "MODIFY COLUMN";
case AlterCommand::Type::MODIFY_ORDER_BY:
return "MODIFY ORDER BY";
case AlterCommand::Type::MODIFY_SAMPLE_BY:
return "MODIFY SAMPLE BY";
case AlterCommand::Type::MODIFY_TTL:
return "MODIFY TTL";
case AlterCommand::Type::MODIFY_SETTING:
return "MODIFY SETTING";
case AlterCommand::Type::RESET_SETTING:
return "RESET SETTING";
case AlterCommand::Type::MODIFY_QUERY:
return "MODIFY QUERY";
case AlterCommand::Type::RENAME_COLUMN:
return "RENAME COLUMN";
case AlterCommand::Type::REMOVE_TTL:
return "REMOVE TTL";
default:
throw Exception("Uninitialized ALTER command", ErrorCodes::LOGICAL_ERROR);
}
}
void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const void AlterCommands::apply(StorageInMemoryMetadata & metadata, ContextPtr context) const
{ {
if (!prepared) if (!prepared)

View File

@ -168,9 +168,6 @@ struct AlterCommand
std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const; std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const;
}; };
/// Return string representation of AlterCommand::Type
String alterTypeToString(const AlterCommand::Type type);
class Context; class Context;
/// Vector of AlterCommand with several additional functions /// Vector of AlterCommand with several additional functions

View File

@ -140,9 +140,8 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (!command.isCommentAlter()) if (!command.isCommentAlter())
throw Exception( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), command.type, getName());
ErrorCodes::NOT_IMPLEMENTED);
} }
} }

View File

@ -480,39 +480,16 @@ UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const
return res; return res;
} }
String IMergeTreeDataPart::stateToString(IMergeTreeDataPart::State state)
{
switch (state)
{
case State::Temporary:
return "Temporary";
case State::PreCommitted:
return "PreCommitted";
case State::Committed:
return "Committed";
case State::Outdated:
return "Outdated";
case State::Deleting:
return "Deleting";
case State::DeleteOnDestroy:
return "DeleteOnDestroy";
}
__builtin_unreachable();
}
String IMergeTreeDataPart::stateString() const
{
return stateToString(state);
}
void IMergeTreeDataPart::assertState(const std::initializer_list<IMergeTreeDataPart::State> & affordable_states) const void IMergeTreeDataPart::assertState(const std::initializer_list<IMergeTreeDataPart::State> & affordable_states) const
{ {
if (!checkState(affordable_states)) if (!checkState(affordable_states))
{ {
String states_str; String states_str;
for (auto affordable_state : affordable_states) for (auto affordable_state : affordable_states)
states_str += stateToString(affordable_state) + " "; {
states_str += stateString(affordable_state);
states_str += ' ';
}
throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART); throw Exception("Unexpected state of part " + getNameWithState() + ". Expected: " + states_str, ErrorCodes::NOT_FOUND_EXPECTED_DATA_PART);
} }

View File

@ -16,7 +16,6 @@
#include <shared_mutex> #include <shared_mutex>
namespace zkutil namespace zkutil
{ {
class ZooKeeper; class ZooKeeper;
@ -221,12 +220,6 @@ public:
DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor
}; };
static constexpr auto all_part_states =
{
State::Temporary, State::PreCommitted, State::Committed, State::Outdated, State::Deleting,
State::DeleteOnDestroy
};
using TTLInfo = MergeTreeDataPartTTLInfo; using TTLInfo = MergeTreeDataPartTTLInfo;
using TTLInfos = MergeTreeDataPartTTLInfos; using TTLInfos = MergeTreeDataPartTTLInfos;
@ -236,14 +229,10 @@ public:
void setState(State new_state) const; void setState(State new_state) const;
State getState() const; State getState() const;
/// Returns name of state static constexpr std::string_view stateString(State state) { return magic_enum::enum_name(state); }
static String stateToString(State state); constexpr std::string_view stateString() const { return stateString(state); }
String stateString() const;
String getNameWithState() const String getNameWithState() const { return fmt::format("{} (state {})", name, stateString()); }
{
return name + " (state " + stateString() + ")";
}
/// Returns true if state of part is one of affordable_states /// Returns true if state of part is one of affordable_states
bool checkState(const std::initializer_list<State> & affordable_states) const bool checkState(const std::initializer_list<State> & affordable_states) const

View File

@ -163,7 +163,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
LOG_DEBUG( LOG_DEBUG(
log, log,
"Choose {} projection {}", "Choose {} projection {}",
ProjectionDescription::typeToString(query_info.projection->desc->type), query_info.projection->desc->type,
query_info.projection->desc->name); query_info.projection->desc->name);
Pipes pipes; Pipes pipes;

View File

@ -27,20 +27,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
}; };
const char * ProjectionDescription::typeToString(Type type)
{
switch (type)
{
case Type::Normal:
return "normal";
case Type::Aggregate:
return "aggregate";
}
__builtin_unreachable();
}
bool ProjectionDescription::isPrimaryKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const bool ProjectionDescription::isPrimaryKeyColumnPossiblyWrappedInFunctions(const ASTPtr & node) const
{ {
const String column_name = node->getColumnName(); const String column_name = node->getColumnName();

View File

@ -30,8 +30,6 @@ struct ProjectionDescription
static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection"; static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection";
static const char * typeToString(Type type);
/// Definition AST of projection /// Definition AST of projection
ASTPtr definition_ast; ASTPtr definition_ast;

View File

@ -1021,11 +1021,13 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context
auto name_deps = getDependentViewsByColumn(local_context); auto name_deps = getDependentViewsByColumn(local_context);
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN if (command.type != AlterCommand::Type::ADD_COLUMN
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN) && command.type != AlterCommand::Type::MODIFY_COLUMN
throw Exception( && command.type != AlterCommand::Type::DROP_COLUMN
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), && command.type != AlterCommand::Type::COMMENT_COLUMN)
ErrorCodes::NOT_IMPLEMENTED); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
command.type, getName());
if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear) if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear)
{ {
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];

View File

@ -785,8 +785,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co
&& command.type != AlterCommand::Type::COMMENT_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN
&& command.type != AlterCommand::Type::RENAME_COLUMN) && command.type != AlterCommand::Type::RENAME_COLUMN)
throw Exception("Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
ErrorCodes::NOT_IMPLEMENTED); command.type, getName());
if (command.type == AlterCommand::DROP_COLUMN && !command.clear) if (command.type == AlterCommand::DROP_COLUMN && !command.clear)
{ {
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];

View File

@ -308,9 +308,8 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_QUERY) if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_QUERY)
throw Exception( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), command.type, getName());
ErrorCodes::NOT_IMPLEMENTED);
} }
} }
else else
@ -318,9 +317,8 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (!command.isCommentAlter()) if (!command.isCommentAlter())
throw Exception( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), command.type, getName());
ErrorCodes::NOT_IMPLEMENTED);
} }
} }
} }

View File

@ -625,11 +625,13 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP
auto name_deps = getDependentViewsByColumn(local_context); auto name_deps = getDependentViewsByColumn(local_context);
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN if (command.type != AlterCommand::Type::ADD_COLUMN
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN) && command.type != AlterCommand::Type::MODIFY_COLUMN
throw Exception( && command.type != AlterCommand::Type::DROP_COLUMN
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), && command.type != AlterCommand::Type::COMMENT_COLUMN)
ErrorCodes::NOT_IMPLEMENTED); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
command.type, getName());
if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear) if (command.type == AlterCommand::Type::DROP_COLUMN && !command.clear)
{ {
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];

View File

@ -41,11 +41,13 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt
auto name_deps = getDependentViewsByColumn(context); auto name_deps = getDependentViewsByColumn(context);
for (const auto & command : commands) for (const auto & command : commands)
{ {
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN if (command.type != AlterCommand::Type::ADD_COLUMN
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN) && command.type != AlterCommand::Type::MODIFY_COLUMN
throw Exception( && command.type != AlterCommand::Type::DROP_COLUMN
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(), && command.type != AlterCommand::Type::COMMENT_COLUMN)
ErrorCodes::NOT_IMPLEMENTED); throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
command.type, getName());
if (command.type == AlterCommand::DROP_COLUMN && !command.clear) if (command.type == AlterCommand::DROP_COLUMN && !command.clear)
{ {
const auto & deps_mv = name_deps[command.column_name]; const auto & deps_mv = name_deps[command.column_name];

View File

@ -5098,7 +5098,7 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
auto metadata_snapshot = getInMemoryMetadataPtr(); auto metadata_snapshot = getInMemoryMetadataPtr();
const DataPartsVector all_parts = getDataPartsVector(IMergeTreeDataPart::all_part_states); const DataPartsVector all_parts = getAllDataPartsVector();
Strings active_parts_names; Strings active_parts_names;
/// Why all parts (not only Committed) are moved to detached/: /// Why all parts (not only Committed) are moved to detached/:

View File

@ -265,7 +265,7 @@ void StorageSystemParts::processNextStorage(
/// _state column should be the latest. /// _state column should be the latest.
/// Do not use part->getState*, it can be changed from different thread /// Do not use part->getState*, it can be changed from different thread
if (has_state_column) if (has_state_column)
columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state)); columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state));
} }
} }

View File

@ -282,7 +282,7 @@ void StorageSystemProjectionParts::processNextStorage(
/// _state column should be the latest. /// _state column should be the latest.
/// Do not use part->getState*, it can be changed from different thread /// Do not use part->getState*, it can be changed from different thread
if (has_state_column) if (has_state_column)
columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state)); columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state));
} }
} }

View File

@ -6,6 +6,7 @@
#include <DataTypes/DataTypeNullable.h> #include <DataTypes/DataTypeNullable.h>
#include <Processors/Sources/SourceFromSingleChunk.h> #include <Processors/Sources/SourceFromSingleChunk.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <common/EnumReflection.h>
namespace DB namespace DB
@ -68,7 +69,7 @@ Pipe StorageSystemStoragePolicies::read(
for (const auto & disk_ptr : volumes[i]->getDisks()) for (const auto & disk_ptr : volumes[i]->getDisks())
disks.push_back(disk_ptr->getName()); disks.push_back(disk_ptr->getName());
col_disks->insert(disks); col_disks->insert(disks);
col_volume_type->insert(volumeTypeToString(volumes[i]->getType())); col_volume_type->insert(magic_enum::enum_name(volumes[i]->getType()));
col_max_part_size->insert(volumes[i]->max_data_part_size); col_max_part_size->insert(volumes[i]->max_data_part_size);
col_move_factor->insert(policy_ptr->getMoveFactor()); col_move_factor->insert(policy_ptr->getMoveFactor());
col_prefer_not_to_merge->insert(volumes[i]->areMergesAvoided() ? 1 : 0); col_prefer_not_to_merge->insert(volumes[i]->areMergesAvoided() ? 1 : 0);