mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Initial: replacing hardcoded toString for enums with magic_enum
This commit is contained in:
parent
b40b4b0b4d
commit
8e9aacadd1
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -246,3 +246,6 @@
|
||||
[submodule "contrib/bzip2"]
|
||||
path = contrib/bzip2
|
||||
url = https://github.com/ClickHouse-Extras/bzip2.git
|
||||
[submodule "contrib/magic_enum"]
|
||||
path = contrib/magic_enum
|
||||
url = https://github.com/Neargye/magic_enum
|
||||
|
32
base/common/EnumReflection.h
Normal file
32
base/common/EnumReflection.h
Normal 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;
|
||||
}
|
||||
}
|
2
contrib/CMakeLists.txt
vendored
2
contrib/CMakeLists.txt
vendored
@ -32,6 +32,8 @@ endif()
|
||||
|
||||
set_property(DIRECTORY PROPERTY EXCLUDE_FROM_ALL 1)
|
||||
|
||||
include_directories("${CMAKE_SOURCE_DIR}/contrib/magic_enum/include")
|
||||
|
||||
add_subdirectory (abseil-cpp-cmake)
|
||||
add_subdirectory (boost-cmake)
|
||||
add_subdirectory (cctz-cmake)
|
||||
|
1
contrib/magic_enum
vendored
Submodule
1
contrib/magic_enum
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit 38f86e4d093cfc9034a140d37de2168e3951bef3
|
@ -506,8 +506,9 @@ static void pushBackAndCreateState(ColumnAggregateFunction::Container & data, Ar
|
||||
void ColumnAggregateFunction::insert(const Field & x)
|
||||
{
|
||||
if (x.getType() != Field::Types::AggregateFunctionState)
|
||||
throw Exception(String("Inserting field of type ") + x.getTypeName() + " into ColumnAggregateFunction. "
|
||||
"Expected " + Field::Types::toString(Field::Types::AggregateFunctionState), ErrorCodes::LOGICAL_ERROR);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Inserting field of type {} into ColumnAggregateFunction. Expected {}",
|
||||
x.getTypeName(), Field::Types::AggregateFunctionState);
|
||||
|
||||
const auto & field_name = x.get<const AggregateFunctionStateData &>().name;
|
||||
if (type_string != field_name)
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -4,10 +4,11 @@
|
||||
#include <utility>
|
||||
#include <ostream>
|
||||
#include <common/types.h>
|
||||
#include <common/EnumReflection.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
enum class ExternalLoaderStatus
|
||||
enum class ExternalLoaderStatus : Int8
|
||||
{
|
||||
NOT_LOADED, /// Object hasn't been tried to load. This is an initial state.
|
||||
LOADED, /// Object has been loaded successfully.
|
||||
@ -18,7 +19,14 @@ namespace DB
|
||||
NOT_EXIST, /// Object with this name wasn't found in the configuration.
|
||||
};
|
||||
|
||||
String toString(ExternalLoaderStatus status);
|
||||
std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues();
|
||||
std::ostream & operator<<(std::ostream & out, ExternalLoaderStatus status);
|
||||
std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues()
|
||||
{
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
@ -9,23 +9,6 @@ namespace ErrorCodes
|
||||
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
|
||||
{
|
||||
switch (kind)
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <common/types.h>
|
||||
|
||||
#include <common/EnumReflection.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,7 +24,7 @@ struct IntervalKind
|
||||
IntervalKind(Kind kind_ = Second) : kind(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.
|
||||
/// For `Month`, `Quarter` and `Year` the function returns an average number of seconds.
|
||||
|
@ -33,7 +33,7 @@ using ItemPtr = std::unique_ptr<IItem>;
|
||||
class JSONString : public IItem
|
||||
{
|
||||
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;
|
||||
|
||||
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, 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, 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))); }
|
||||
|
||||
template <typename T, std::enable_if_t<std::is_arithmetic<T>::value, bool> = true>
|
||||
|
@ -15,7 +15,6 @@
|
||||
#include <common/DayNum.h>
|
||||
#include <common/strong_typedef.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -283,33 +282,6 @@ public:
|
||||
Map = 26,
|
||||
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; }
|
||||
|
||||
//static constexpr std::string_view getTypeName() { return Types::toString(which); }
|
||||
const char * getTypeName() const { return Types::toString(which); }
|
||||
|
||||
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;
|
||||
if (target != 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
|
||||
|
||||
StoredType * MAY_ALIAS ptr = reinterpret_cast<StoredType *>(&storage);
|
||||
@ -821,8 +796,11 @@ template <typename T>
|
||||
auto & Field::safeGet()
|
||||
{
|
||||
const Types::Which requested = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
|
||||
|
||||
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>();
|
||||
}
|
||||
|
||||
|
@ -27,7 +27,7 @@ public:
|
||||
|
||||
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"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Interval; }
|
||||
|
||||
|
@ -26,10 +26,8 @@ String getExceptionMessage(
|
||||
const String & message, size_t argument_index, const char * argument_name,
|
||||
const std::string & context_data_type_name, Field::Types::Which field_type)
|
||||
{
|
||||
return std::string("Parameter #") + std::to_string(argument_index) + " '"
|
||||
+ argument_name + "' for " + context_data_type_name
|
||||
+ message
|
||||
+ ", expected: " + Field::Types::toString(field_type) + " literal.";
|
||||
return fmt::format("Parameter #{} '{}' for {}{}, expected {} literal",
|
||||
argument_index, argument_name, context_data_type_name, message, field_type);
|
||||
}
|
||||
|
||||
template <typename T, ArgumentKind Kind>
|
||||
|
@ -15,22 +15,6 @@ namespace ErrorCodes
|
||||
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(
|
||||
String name_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
|
@ -16,8 +16,6 @@ enum class VolumeType
|
||||
UNKNOWN
|
||||
};
|
||||
|
||||
String volumeTypeToString(VolumeType t);
|
||||
|
||||
class IVolume;
|
||||
using VolumePtr = std::shared_ptr<IVolume>;
|
||||
using Volumes = std::vector<VolumePtr>;
|
||||
|
@ -632,7 +632,9 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
std::string function_name;
|
||||
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
|
||||
{
|
||||
|
@ -261,10 +261,10 @@ public:
|
||||
+ ". This argument is optional and must be a constant string with timezone name",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
if (first_argument_is_date && result_type_is_date)
|
||||
throw Exception(
|
||||
"The timezone argument of function " + getName() + " with interval type " + interval_type->getKind().toString()
|
||||
+ " is allowed only when the 1st argument has the type DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument "
|
||||
"has the type DateTime",
|
||||
getName(), interval_type->getKind().toString());
|
||||
};
|
||||
|
||||
if (arguments.size() == 2)
|
||||
|
@ -6,6 +6,9 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <functional>
|
||||
#include <type_traits>
|
||||
|
||||
#include <common/EnumReflection.h>
|
||||
|
||||
|
||||
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 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; }
|
||||
/// 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; }
|
||||
template <> inline WriteBuffer & operator<< (WriteBuffer & buf, const std::string_view & x) { writeString(StringRef(x), 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, auto x) requires(std::is_enum_v<decltype(x)>)
|
||||
{
|
||||
writeString(magic_enum::enum_name(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 QuoteManipWriteBuffer operator<< (WriteBuffer & buf, QuoteManip) { return buf; }
|
||||
|
@ -42,7 +42,6 @@
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
@ -177,17 +176,20 @@ inline void writeFloatText(T x, WriteBuffer & buf)
|
||||
buf.write(buffer, result);
|
||||
}
|
||||
|
||||
|
||||
inline void writeString(const char * data, size_t size, WriteBuffer & buf)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
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.
|
||||
* Use when the string is a literal.
|
||||
|
@ -30,34 +30,15 @@ namespace ErrorCodes
|
||||
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
|
||||
{
|
||||
map.add("Node Type", ActionsDAG::typeToString(type));
|
||||
map.add("Node Type", magic_enum::enum_name(type));
|
||||
|
||||
if (result_type)
|
||||
map.add("Result Type", result_type->getName());
|
||||
|
||||
if (!result_name.empty())
|
||||
map.add("Result Type", ActionsDAG::typeToString(type));
|
||||
map.add("Result Type", magic_enum::enum_name(type));
|
||||
|
||||
if (column)
|
||||
map.add("Column", column->getName());
|
||||
|
@ -61,8 +61,6 @@ public:
|
||||
FUNCTION,
|
||||
};
|
||||
|
||||
static const char * typeToString(ActionType type);
|
||||
|
||||
struct Node;
|
||||
using NodeRawPtrs = std::vector<Node *>;
|
||||
using NodeRawConstPtrs = std::vector<const Node *>;
|
||||
|
@ -624,7 +624,7 @@ void makeWindowDescriptionFromAST(const Context & context,
|
||||
{
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Window frame '{}' is not implemented (while processing '{}')",
|
||||
WindowFrame::toString(definition.frame_type),
|
||||
definition.frame_type,
|
||||
ast->formatForErrorMessage());
|
||||
}
|
||||
|
||||
|
@ -426,7 +426,7 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
}
|
||||
case Type::STOP_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:
|
||||
throw Exception("Unknown type of SYSTEM query", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ std::string WindowFrame::toString() const
|
||||
|
||||
void WindowFrame::toString(WriteBuffer & buf) const
|
||||
{
|
||||
buf << toString(type) << " BETWEEN ";
|
||||
buf << type << " BETWEEN ";
|
||||
if (begin_type == BoundaryType::Current)
|
||||
{
|
||||
buf << "CURRENT ROW";
|
||||
@ -101,9 +101,9 @@ void WindowFrame::checkValid() const
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Frame start offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
|
||||
toString(type),
|
||||
type,
|
||||
applyVisitor(FieldVisitorToString(), begin_offset),
|
||||
Field::Types::toString(begin_offset.getType()));
|
||||
begin_offset.getType());
|
||||
}
|
||||
|
||||
if (end_type == BoundaryType::Offset
|
||||
@ -114,9 +114,9 @@ void WindowFrame::checkValid() const
|
||||
{
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Frame end offset for '{}' frame must be a nonnegative 32-bit integer, '{}' of type '{}' given",
|
||||
toString(type),
|
||||
type,
|
||||
applyVisitor(FieldVisitorToString(), end_offset),
|
||||
Field::Types::toString(end_offset.getType()));
|
||||
end_offset.getType());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -73,40 +73,6 @@ struct WindowFrame
|
||||
&& 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
|
||||
|
@ -75,8 +75,8 @@ static Field convertNumericType(const Field & from, const IDataType & type)
|
||||
if (from.getType() == Field::Types::Int256)
|
||||
return convertNumericTypeImpl<Int256, To>(from);
|
||||
|
||||
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: "
|
||||
+ Field::Types::toString(from.getType()), ErrorCodes::TYPE_MISMATCH);
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
|
||||
type.getName(), from.getType());
|
||||
}
|
||||
|
||||
|
||||
@ -134,8 +134,8 @@ static Field convertDecimalType(const Field & from, const To & type)
|
||||
if (from.getType() == Field::Types::Decimal128)
|
||||
return convertDecimalToDecimalType<Decimal128>(from, type);
|
||||
|
||||
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: "
|
||||
+ Field::Types::toString(from.getType()), ErrorCodes::TYPE_MISMATCH);
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
|
||||
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);
|
||||
}
|
||||
|
||||
throw Exception("Type mismatch in IN or VALUES section. Expected: " + type.getName() + ". Got: "
|
||||
+ Field::Types::toString(src.getType()), ErrorCodes::TYPE_MISMATCH);
|
||||
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in IN or VALUES section. Expected: {}. Got: {}",
|
||||
type.getName(), src.getType());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,101 +13,10 @@ namespace ErrorCodes
|
||||
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
|
||||
{
|
||||
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 = [&]
|
||||
{
|
||||
|
@ -62,8 +62,6 @@ public:
|
||||
END
|
||||
};
|
||||
|
||||
static const char * typeToString(Type type);
|
||||
|
||||
Type type = Type::UNKNOWN;
|
||||
|
||||
String target_model;
|
||||
|
@ -98,7 +98,7 @@ void ASTWindowDefinition::formatImpl(const FormatSettings & settings,
|
||||
settings.ostr << " ";
|
||||
}
|
||||
|
||||
settings.ostr << WindowFrame::toString(frame_type) << " BETWEEN ";
|
||||
settings.ostr << frame_type << " BETWEEN ";
|
||||
if (frame_begin_type == WindowFrame::BoundaryType::Current)
|
||||
{
|
||||
settings.ostr << "CURRENT ROW";
|
||||
|
@ -14,31 +14,18 @@ namespace ErrorCodes
|
||||
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)
|
||||
{
|
||||
if (pos->type != TokenType::BareWord)
|
||||
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)
|
||||
throw Exception("Logical error: keyword cannot be empty string", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const char * s_end = s + s_length;
|
||||
|
||||
while (true)
|
||||
{
|
||||
expected.add(pos, current_word);
|
||||
|
@ -13,13 +13,13 @@ namespace DB
|
||||
class ParserKeyword : public IParserBase
|
||||
{
|
||||
private:
|
||||
const char * s;
|
||||
std::string_view s;
|
||||
|
||||
public:
|
||||
ParserKeyword(const char * s_);
|
||||
constexpr ParserKeyword(std::string_view s_): s(s_) {}
|
||||
|
||||
protected:
|
||||
const char * getName() const override;
|
||||
const char * getName() const override { return s.data(); }
|
||||
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
|
||||
};
|
||||
|
@ -6,6 +6,8 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/parseDatabaseAndTableName.h>
|
||||
|
||||
#include <span>
|
||||
#include <common/EnumReflection.h>
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
@ -67,13 +69,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
|
||||
auto res = std::make_shared<ASTSystemQuery>();
|
||||
|
||||
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{ASTSystemQuery::typeToString(t)}.ignore(pos, expected))
|
||||
if (ParserKeyword(UnderscoreToSpace(str)).ignore(pos, expected))
|
||||
{
|
||||
res->type = t;
|
||||
res->type = entry;
|
||||
found = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -617,8 +617,8 @@ void WindowTransform::advanceFrameStart()
|
||||
default:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"Frame start type '{}' for frame '{}' is not implemented",
|
||||
WindowFrame::toString(window_description.frame.begin_type),
|
||||
WindowFrame::toString(window_description.frame.type));
|
||||
window_description.frame.begin_type,
|
||||
window_description.frame.type);
|
||||
}
|
||||
break;
|
||||
}
|
||||
@ -849,7 +849,7 @@ void WindowTransform::advanceFrameEnd()
|
||||
default:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
||||
"The frame end type '{}' is not implemented",
|
||||
WindowFrame::toString(window_description.frame.end_type));
|
||||
window_description.frame.end_type);
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
@ -842,53 +842,6 @@ std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(Storage
|
||||
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
|
||||
{
|
||||
if (!prepared)
|
||||
|
@ -168,9 +168,6 @@ struct AlterCommand
|
||||
std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, ContextPtr context) const;
|
||||
};
|
||||
|
||||
/// Return string representation of AlterCommand::Type
|
||||
String alterTypeToString(const AlterCommand::Type type);
|
||||
|
||||
class Context;
|
||||
|
||||
/// Vector of AlterCommand with several additional functions
|
||||
|
@ -140,9 +140,8 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, ContextPtr /
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (!command.isCommentAlter())
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
||||
command.type, getName());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -480,39 +480,16 @@ UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const
|
||||
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
|
||||
{
|
||||
if (!checkState(affordable_states))
|
||||
{
|
||||
String states_str;
|
||||
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);
|
||||
}
|
||||
|
@ -16,7 +16,6 @@
|
||||
|
||||
#include <shared_mutex>
|
||||
|
||||
|
||||
namespace zkutil
|
||||
{
|
||||
class ZooKeeper;
|
||||
@ -221,12 +220,6 @@ public:
|
||||
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 TTLInfos = MergeTreeDataPartTTLInfos;
|
||||
|
||||
@ -236,14 +229,10 @@ public:
|
||||
void setState(State new_state) const;
|
||||
State getState() const;
|
||||
|
||||
/// Returns name of state
|
||||
static String stateToString(State state);
|
||||
String stateString() const;
|
||||
static constexpr std::string_view stateString(State state) { return magic_enum::enum_name(state); }
|
||||
constexpr std::string_view stateString() const { return stateString(state); }
|
||||
|
||||
String getNameWithState() const
|
||||
{
|
||||
return name + " (state " + stateString() + ")";
|
||||
}
|
||||
String getNameWithState() const { return fmt::format("{} (state {})", name, stateString()); }
|
||||
|
||||
/// Returns true if state of part is one of affordable_states
|
||||
bool checkState(const std::initializer_list<State> & affordable_states) const
|
||||
|
@ -163,7 +163,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
|
||||
LOG_DEBUG(
|
||||
log,
|
||||
"Choose {} projection {}",
|
||||
ProjectionDescription::typeToString(query_info.projection->desc->type),
|
||||
query_info.projection->desc->type,
|
||||
query_info.projection->desc->name);
|
||||
|
||||
Pipes pipes;
|
||||
|
@ -27,20 +27,6 @@ namespace ErrorCodes
|
||||
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
|
||||
{
|
||||
const String column_name = node->getColumnName();
|
||||
|
@ -30,8 +30,6 @@ struct ProjectionDescription
|
||||
|
||||
static constexpr const char * MINMAX_COUNT_PROJECTION_NAME = "_minmax_count_projection";
|
||||
|
||||
static const char * typeToString(Type type);
|
||||
|
||||
/// Definition AST of projection
|
||||
ASTPtr definition_ast;
|
||||
|
||||
|
@ -1021,11 +1021,13 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, Context
|
||||
auto name_deps = getDependentViewsByColumn(local_context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN
|
||||
&& command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN
|
||||
&& command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
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)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
|
@ -785,8 +785,9 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, Co
|
||||
&& command.type != AlterCommand::Type::COMMENT_COLUMN
|
||||
&& command.type != AlterCommand::Type::RENAME_COLUMN)
|
||||
|
||||
throw Exception("Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
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)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
|
@ -308,9 +308,8 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (!command.isCommentAlter() && command.type != AlterCommand::MODIFY_QUERY)
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
||||
command.type, getName());
|
||||
}
|
||||
}
|
||||
else
|
||||
@ -318,9 +317,8 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (!command.isCommentAlter())
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
||||
command.type, getName());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -625,11 +625,13 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, ContextP
|
||||
auto name_deps = getDependentViewsByColumn(local_context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN
|
||||
&& command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN
|
||||
&& command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
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)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
|
@ -41,11 +41,13 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt
|
||||
auto name_deps = getDependentViewsByColumn(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN && command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
throw Exception(
|
||||
"Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
if (command.type != AlterCommand::Type::ADD_COLUMN
|
||||
&& command.type != AlterCommand::Type::MODIFY_COLUMN
|
||||
&& command.type != AlterCommand::Type::DROP_COLUMN
|
||||
&& command.type != AlterCommand::Type::COMMENT_COLUMN)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Alter of type '{}' is not supported by storage {}",
|
||||
command.type, getName());
|
||||
|
||||
if (command.type == AlterCommand::DROP_COLUMN && !command.clear)
|
||||
{
|
||||
const auto & deps_mv = name_deps[command.column_name];
|
||||
|
@ -5098,7 +5098,7 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper()
|
||||
|
||||
auto metadata_snapshot = getInMemoryMetadataPtr();
|
||||
|
||||
const DataPartsVector all_parts = getDataPartsVector(IMergeTreeDataPart::all_part_states);
|
||||
const DataPartsVector all_parts = getAllDataPartsVector();
|
||||
Strings active_parts_names;
|
||||
|
||||
/// Why all parts (not only Committed) are moved to detached/:
|
||||
|
@ -265,7 +265,7 @@ void StorageSystemParts::processNextStorage(
|
||||
/// _state column should be the latest.
|
||||
/// Do not use part->getState*, it can be changed from different thread
|
||||
if (has_state_column)
|
||||
columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state));
|
||||
columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -282,7 +282,7 @@ void StorageSystemProjectionParts::processNextStorage(
|
||||
/// _state column should be the latest.
|
||||
/// Do not use part->getState*, it can be changed from different thread
|
||||
if (has_state_column)
|
||||
columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state));
|
||||
columns[res_index++]->insert(IMergeTreeDataPart::stateString(part_state));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <common/EnumReflection.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -68,7 +69,7 @@ Pipe StorageSystemStoragePolicies::read(
|
||||
for (const auto & disk_ptr : volumes[i]->getDisks())
|
||||
disks.push_back(disk_ptr->getName());
|
||||
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_move_factor->insert(policy_ptr->getMoveFactor());
|
||||
col_prefer_not_to_merge->insert(volumes[i]->areMergesAvoided() ? 1 : 0);
|
||||
|
Loading…
Reference in New Issue
Block a user