Simplifying exception messages

This commit is contained in:
Mike Kot 2021-09-06 17:59:46 +02:00
parent 8e9aacadd1
commit c4385a2c40
40 changed files with 244 additions and 458 deletions

View File

@ -85,6 +85,7 @@ target_link_libraries (common
replxx
cctz
fmt
magic_enum
)
if (ENABLE_TESTS)

17
base/common/Concepts.h Normal file
View File

@ -0,0 +1,17 @@
#pragma once
#include <type_traits>
#include <concepts>
#include "extended_types.h"
namespace DB
{
// Concept for enums can't be named just "enum", so they all start with is_ for consistency
/// C++ integral types as stated in standard + wide integer types [U]Int[128, 256]
template <class T> concept is_integer = is_integer_v<T>;
template <class T> concept is_floating_point = std::is_floating_point_v<T>;
template <class T> concept is_enum = std::is_enum_v<T>;
}

View File

@ -1,32 +1,16 @@
#pragma once
#include <type_traits>
#include <magic_enum.hpp>
#include <fmt/format.h>
#include <../../contrib/magic_enum/include/magic_enum.hpp> //FIXME
#include "Concepts.h"
// 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) {
/// Enable printing enum values as strings via fmt + magic_enum
template <DB::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

@ -36,18 +36,7 @@
namespace detail
{
template <char s0>
inline bool is_in(char x)
{
return x == s0;
}
template <char s0, char s1, char... tail>
inline bool is_in(char x)
{
return x == s0 || is_in<s1, tail...>(x);
}
template <char ...chars> constexpr bool is_in(char x) { return ((x == chars) || ...); }
#if defined(__SSE2__)
template <char s0>
@ -67,16 +56,10 @@ inline __m128i mm_is_in(__m128i bytes)
#endif
template <bool positive>
bool maybe_negate(bool x)
{
if constexpr (positive)
return x;
else
return !x;
}
constexpr bool maybe_negate(bool x) { return x == positive; }
template <bool positive>
uint16_t maybe_negate(uint16_t x)
constexpr uint16_t maybe_negate(uint16_t x)
{
if constexpr (positive)
return x;
@ -84,11 +67,7 @@ uint16_t maybe_negate(uint16_t x)
return ~x;
}
enum class ReturnMode
{
End,
Nullptr,
};
enum class ReturnMode { End, Nullptr };
template <bool positive, ReturnMode return_mode, char... symbols>
@ -149,12 +128,13 @@ template <bool positive, ReturnMode return_mode, size_t num_chars,
char c05 = 0, char c06 = 0, char c07 = 0, char c08 = 0,
char c09 = 0, char c10 = 0, char c11 = 0, char c12 = 0,
char c13 = 0, char c14 = 0, char c15 = 0, char c16 = 0>
inline const char * find_first_symbols_sse42_impl(const char * const begin, const char * const end)
inline const char * find_first_symbols_sse42(const char * const begin, const char * const end)
{
const char * pos = begin;
#if defined(__SSE4_2__)
#define MODE (_SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_ANY | _SIDD_LEAST_SIGNIFICANT)
constexpr int mode = _SIDD_UBYTE_OPS | _SIDD_CMP_EQUAL_ANY | _SIDD_LEAST_SIGNIFICANT;
__m128i set = _mm_setr_epi8(c01, c02, c03, c04, c05, c06, c07, c08, c09, c10, c11, c12, c13, c14, c15, c16);
for (; pos + 15 < end; pos += 16)
@ -163,16 +143,15 @@ inline const char * find_first_symbols_sse42_impl(const char * const begin, cons
if constexpr (positive)
{
if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE))
return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE);
if (_mm_cmpestrc(set, num_chars, bytes, 16, mode))
return pos + _mm_cmpestri(set, num_chars, bytes, 16, mode);
}
else
{
if (_mm_cmpestrc(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY))
return pos + _mm_cmpestri(set, num_chars, bytes, 16, MODE | _SIDD_NEGATIVE_POLARITY);
if (_mm_cmpestrc(set, num_chars, bytes, 16, mode | _SIDD_NEGATIVE_POLARITY))
return pos + _mm_cmpestri(set, num_chars, bytes, 16, mode | _SIDD_NEGATIVE_POLARITY);
}
}
#undef MODE
#endif
for (; pos < end; ++pos)
@ -197,28 +176,23 @@ inline const char * find_first_symbols_sse42_impl(const char * const begin, cons
}
template <bool positive, ReturnMode return_mode, char... symbols>
inline const char * find_first_symbols_sse42(const char * begin, const char * end)
{
return find_first_symbols_sse42_impl<positive, return_mode, sizeof...(symbols), symbols...>(begin, end);
}
/// NOTE No SSE 4.2 implementation for find_last_symbols_or_null. Not worth to do.
template <bool positive, ReturnMode return_mode, char... symbols>
inline const char * find_first_symbols_dispatch(const char * begin, const char * end)
{
static_assert(sizeof...(symbols) <= 16, "Symbols mask max size is 16");
static_assert(sizeof...(symbols) > 0, "Symbols mask cannot be empty");
#if defined(__SSE4_2__)
if (sizeof...(symbols) >= 5)
return find_first_symbols_sse42<positive, return_mode, symbols...>(begin, end);
return find_first_symbols_sse42<positive, return_mode, sizeof...(symbols), symbols...>(begin, end);
else
#endif
return find_first_symbols_sse2<positive, return_mode, symbols...>(begin, end);
}
}
template <char... symbols>
inline const char * find_first_symbols(const char * begin, const char * end)
{

View File

@ -32,9 +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 (magic-enum-cmake)
add_subdirectory (boost-cmake)
add_subdirectory (cctz-cmake)
add_subdirectory (consistent-hashing)

View File

@ -0,0 +1,3 @@
set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/magic_enum")
add_library (magic_enum INTERFACE)
target_include_directories(magic_enum INTERFACE ${LIBRARY_DIR}/include)

View File

@ -159,6 +159,7 @@ function clone_submodules
cd "$FASTTEST_SOURCE"
SUBMODULES_TO_UPDATE=(
contrib/magic_enum
contrib/abseil-cpp
contrib/boost
contrib/zlib-ng

View File

@ -494,7 +494,6 @@
M(523, UNKNOWN_ROW_POLICY) \
M(524, ALTER_OF_COLUMN_IS_FORBIDDEN) \
M(525, INCORRECT_DISK_INDEX) \
M(526, UNKNOWN_VOLUME_TYPE) \
M(527, NO_SUITABLE_FUNCTION_IMPLEMENTATION) \
M(528, CASSANDRA_INTERNAL_ERROR) \
M(529, NOT_A_LEADER) \

View File

@ -1,8 +1,6 @@
#pragma once
#include <vector>
#include <utility>
#include <ostream>
#include <common/types.h>
#include <common/EnumReflection.h>
@ -19,13 +17,13 @@ namespace DB
NOT_EXIST, /// Object with this name wasn't found in the configuration.
};
std::vector<std::pair<String, Int8>> getStatusEnumAllPossibleValues()
inline 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);
out.emplace_back(std::string{str}, static_cast<Int8>(value));
return out;
}

View File

@ -43,7 +43,7 @@ std::pair<std::string, UInt16> parseAddress(const std::string & str, UInt16 defa
UInt16 port_number;
ReadBufferFromMemory port_buf(port, end - port);
if (!tryReadText<UInt16>(port_number, port_buf) || !port_buf.eof())
if (!tryReadText(port_number, port_buf) || !port_buf.eof())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Illegal port passed to function parseAddress: {}", port);

View File

@ -487,11 +487,11 @@ template bool decimalLessOrEqual<DateTime64>(DateTime64 x, DateTime64 y, UInt32
inline void writeText(const Null & x, WriteBuffer & buf)
{
if (x.isNegativeInfinity())
writeText(std::string("-Inf"), buf);
writeText("-Inf", buf);
if (x.isPositiveInfinity())
writeText(std::string("+Inf"), buf);
writeText("+Inf", buf);
else
writeText(std::string("NULL"), buf);
writeText("NULL", buf);
}
String toString(const Field & x)

View File

@ -14,6 +14,7 @@
#include <Core/UUID.h>
#include <common/DayNum.h>
#include <common/strong_typedef.h>
#include <common/EnumReflection.h>
namespace DB
{
@ -390,8 +391,7 @@ 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); }
constexpr std::string_view getTypeName() const { return magic_enum::enum_name(which); }
bool isNull() const { return which == Types::Null; }
template <typename T>
@ -782,7 +782,7 @@ 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,
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid Field get from type {} to type {}", which, target);
#endif
@ -798,7 +798,7 @@ auto & Field::safeGet()
const Types::Which requested = TypeToEnum<NearestFieldType<std::decay_t<T>>>::value;
if (which != requested)
throw Exception(ErrorCodes::BAD_GET,
throw Exception(ErrorCodes::BAD_GET,
"Bad get: has {}, requested {}", getTypeName(), requested);
return get<T>();

View File

@ -263,53 +263,6 @@ template <> inline constexpr bool OverBigInt<Int256> = true;
template <> inline constexpr bool OverBigInt<UInt256> = true;
template <> inline constexpr bool OverBigInt<Decimal128> = true;
template <> inline constexpr bool OverBigInt<Decimal256> = true;
inline constexpr const char * getTypeName(TypeIndex idx)
{
switch (idx)
{
case TypeIndex::Nothing: return "Nothing";
case TypeIndex::UInt8: return "UInt8";
case TypeIndex::UInt16: return "UInt16";
case TypeIndex::UInt32: return "UInt32";
case TypeIndex::UInt64: return "UInt64";
case TypeIndex::UInt128: return "UInt128";
case TypeIndex::UInt256: return "UInt256";
case TypeIndex::Int8: return "Int8";
case TypeIndex::Int16: return "Int16";
case TypeIndex::Int32: return "Int32";
case TypeIndex::Int64: return "Int64";
case TypeIndex::Int128: return "Int128";
case TypeIndex::Int256: return "Int256";
case TypeIndex::Float32: return "Float32";
case TypeIndex::Float64: return "Float64";
case TypeIndex::Date: return "Date";
case TypeIndex::Date32: return "Date32";
case TypeIndex::DateTime: return "DateTime";
case TypeIndex::DateTime64: return "DateTime64";
case TypeIndex::String: return "String";
case TypeIndex::FixedString: return "FixedString";
case TypeIndex::Enum8: return "Enum8";
case TypeIndex::Enum16: return "Enum16";
case TypeIndex::Decimal32: return "Decimal32";
case TypeIndex::Decimal64: return "Decimal64";
case TypeIndex::Decimal128: return "Decimal128";
case TypeIndex::Decimal256: return "Decimal256";
case TypeIndex::UUID: return "UUID";
case TypeIndex::Array: return "Array";
case TypeIndex::Tuple: return "Tuple";
case TypeIndex::Set: return "Set";
case TypeIndex::Interval: return "Interval";
case TypeIndex::Nullable: return "Nullable";
case TypeIndex::Function: return "Function";
case TypeIndex::AggregateFunction: return "AggregateFunction";
case TypeIndex::LowCardinality: return "LowCardinality";
case TypeIndex::Map: return "Map";
}
__builtin_unreachable();
}
}
/// Specialization of `std::hash` for the Decimal<T> types.

View File

@ -134,7 +134,8 @@ Field DataTypeEnum<Type>::castToName(const Field & value_or_name) const
return this->getNameForValue(static_cast<Type>(value)).toString();
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD,
"DataTypeEnum: Unsupported type of field {}", value_or_name.getTypeName());
}
template <typename Type>
@ -153,7 +154,8 @@ Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
return value;
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
throw Exception(ErrorCodes::BAD_TYPE_OF_FIELD,
"DataTypeEnum: Unsupported type of field {}", value_or_name.getTypeName());
}

View File

@ -47,7 +47,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume
else
{
if (argument && argument->value.getType() != field_type)
throw Exception(getExceptionMessage(String(" has wrong type: ") + argument->value.getTypeName(),
throw Exception(getExceptionMessage(fmt::format(" has wrong type: {}", argument->value.getTypeName()),
argument_index, argument_name, context_data_type_name, field_type), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception(getExceptionMessage(" is missing", argument_index, argument_name, context_data_type_name, field_type),

View File

@ -19,84 +19,37 @@ namespace DB
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int TYPE_MISMATCH;
extern const int BAD_ARGUMENTS;
}
namespace
{
DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_type)
{
const auto name = config.getString(config_prefix + ".name", "");
const auto expression = config.getString(config_prefix + ".expression", "");
DictionaryTypedSpecialAttribute makeDictionaryTypedSpecialAttribute(
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const std::string & default_type)
{
const auto name = config.getString(config_prefix + ".name", "");
const auto expression = config.getString(config_prefix + ".expression", "");
if (name.empty() && !expression.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty");
const auto type_name = config.getString(config_prefix + ".type", default_type);
return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)};
}
if (name.empty() && !expression.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Element {}.name is empty");
const auto type_name = config.getString(config_prefix + ".type", default_type);
return DictionaryTypedSpecialAttribute{std::move(name), std::move(expression), DataTypeFactory::instance().get(type_name)};
}
AttributeUnderlyingType getAttributeUnderlyingType(const DataTypePtr & type)
std::optional<AttributeUnderlyingType> maybeGetAttributeUnderlyingType(TypeIndex index)
{
auto type_index = type->getTypeId();
switch (type_index)
switch (index) /// Special cases which do not map TypeIndex::T -> AttributeUnderlyingType::T
{
case TypeIndex::UInt8: return AttributeUnderlyingType::UInt8;
case TypeIndex::UInt16: return AttributeUnderlyingType::UInt16;
case TypeIndex::UInt32: return AttributeUnderlyingType::UInt32;
case TypeIndex::UInt64: return AttributeUnderlyingType::UInt64;
case TypeIndex::UInt128: return AttributeUnderlyingType::UInt128;
case TypeIndex::UInt256: return AttributeUnderlyingType::UInt256;
case TypeIndex::Int8: return AttributeUnderlyingType::Int8;
case TypeIndex::Int16: return AttributeUnderlyingType::Int16;
case TypeIndex::Int32: return AttributeUnderlyingType::Int32;
case TypeIndex::Int64: return AttributeUnderlyingType::Int64;
case TypeIndex::Int128: return AttributeUnderlyingType::Int128;
case TypeIndex::Int256: return AttributeUnderlyingType::Int256;
case TypeIndex::Float32: return AttributeUnderlyingType::Float32;
case TypeIndex::Float64: return AttributeUnderlyingType::Float64;
case TypeIndex::Decimal32: return AttributeUnderlyingType::Decimal32;
case TypeIndex::Decimal64: return AttributeUnderlyingType::Decimal64;
case TypeIndex::Decimal128: return AttributeUnderlyingType::Decimal128;
case TypeIndex::Decimal256: return AttributeUnderlyingType::Decimal256;
case TypeIndex::Date: return AttributeUnderlyingType::UInt16;
case TypeIndex::DateTime: return AttributeUnderlyingType::UInt32;
case TypeIndex::DateTime64: return AttributeUnderlyingType::UInt64;
case TypeIndex::UUID: return AttributeUnderlyingType::UUID;
case TypeIndex::String: return AttributeUnderlyingType::String;
case TypeIndex::Array: return AttributeUnderlyingType::Array;
case TypeIndex::Date: return AttributeUnderlyingType::UInt16;
case TypeIndex::DateTime: return AttributeUnderlyingType::UInt32;
case TypeIndex::DateTime64: return AttributeUnderlyingType::UInt64;
default: break;
}
throw Exception(ErrorCodes::UNKNOWN_TYPE, "Unknown type {} for dictionary attribute", type->getName());
return magic_enum::enum_cast<AttributeUnderlyingType>(static_cast<TypeIndexUnderlying>(index));
}
std::string toString(AttributeUnderlyingType type)
{
switch (type)
{
#define M(TYPE) case AttributeUnderlyingType::TYPE: return #TYPE;
FOR_ATTRIBUTE_TYPES(M)
#undef M
}
throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Unknown dictionary attribute type {}", toString(static_cast<int>(type)));
}
@ -152,7 +105,7 @@ DictionaryStructure::DictionaryStructure(const Poco::Util::AbstractConfiguration
if (id && attribute.underlying_type != AttributeUnderlyingType::UInt64)
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Hierarchical attribute type for dictionary with simple key must be UInt64. Actual {}",
toString(attribute.underlying_type));
attribute.underlying_type);
else if (key)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Dictionary with complex key does not support hierarchy");
@ -342,7 +295,12 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
bool is_nullable = initial_type->isNullable();
auto non_nullable_type = removeNullable(initial_type);
const auto underlying_type = getAttributeUnderlyingType(non_nullable_type);
const auto underlying_type_opt = maybeGetAttributeUnderlyingType(non_nullable_type->getTypeId());
if (!underlying_type_opt)
throw Exception(ErrorCodes::UNKNOWN_TYPE,
"Unknown type {} for dictionary attribute", non_nullable_type->getName());
const auto expression = config.getString(prefix + "expression", "");
if (!expression.empty())
@ -391,7 +349,7 @@ std::vector<DictionaryAttribute> DictionaryStructure::getAttributes(
res_attributes.emplace_back(DictionaryAttribute{
name,
underlying_type,
*underlying_type_opt,
initial_type,
initial_type_serialization,
expression,

View File

@ -11,53 +11,33 @@
#include <IO/ReadBufferFromString.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/IExternalLoadable.h>
#include <common/EnumReflection.h>
#if defined(__GNUC__)
/// GCC mistakenly warns about the names in enum class.
#pragma GCC diagnostic ignored "-Wshadow"
#endif
#define FOR_ATTRIBUTE_TYPES(M) \
M(UInt8) \
M(UInt16) \
M(UInt32) \
M(UInt64) \
M(UInt128) \
M(UInt256) \
M(Int8) \
M(Int16) \
M(Int32) \
M(Int64) \
M(Int128) \
M(Int256) \
M(Float32) \
M(Float64) \
M(Decimal32) \
M(Decimal64) \
M(Decimal128) \
M(Decimal256) \
M(UUID) \
M(String) \
M(Array) \
namespace DB
{
using TypeIndexUnderlying = magic_enum::underlying_type_t<TypeIndex>;
enum class AttributeUnderlyingType
/// We need to be able to map TypeIndex -> AttributeUnderlyingType and AttributeUnderlyingType -> real type
#define map_item(__T) __T = static_cast<TypeIndexUnderlying>(TypeIndex::__T)
enum class AttributeUnderlyingType : TypeIndexUnderlying
{
#define M(TYPE) TYPE,
FOR_ATTRIBUTE_TYPES(M)
#undef M
map_item(Int8), map_item(Int16), map_item(Int32), map_item(Int64), map_item(Int128), map_item(Int256),
map_item(UInt8), map_item(UInt16), map_item(UInt32), map_item(UInt64), map_item(UInt128), map_item(UInt256),
map_item(Float32), map_item(Float64),
map_item(Decimal32), map_item(Decimal64), map_item(Decimal128), map_item(Decimal256),
map_item(UUID), map_item(String), map_item(Array)
};
#undef map_item
AttributeUnderlyingType getAttributeUnderlyingType(const std::string & type);
std::string toString(AttributeUnderlyingType type);
/// Min and max lifetimes for a dictionary or it's entry
/// Min and max lifetimes for a dictionary or its entry
using DictionaryLifetime = ExternalLoadableLifetime;
/** Holds the description of a single dictionary attribute:
@ -85,24 +65,26 @@ struct DictionaryAttribute final
const bool is_nullable;
};
template <typename Type>
struct DictionaryAttributeType
{
using AttributeType = Type;
};
struct T { using AttributeType = int; };
template <typename F>
void callOnDictionaryAttributeType(AttributeUnderlyingType type, F && func)
{
switch (type)
{
#define M(TYPE) \
case AttributeUnderlyingType::TYPE: \
func(DictionaryAttributeType<TYPE>()); \
break;
FOR_ATTRIBUTE_TYPES(M)
#undef M
}
// for (AttributeUnderlyingType other : magic_enum::enum_values<AttributeUnderlyingType>())
// if (type == other)
// {
//
// }
//
// switch (type)
// {
//#define M(TYPE) \
// case AttributeUnderlyingType::TYPE: \
// func(DictionaryAttributeType<AttributeUnderlyingType::TYPE>()); \
// break;
// FOR_ATTRIBUTE_TYPES(M)
//#undef M
// }
};
struct DictionarySpecialAttribute final
@ -157,7 +139,5 @@ private:
/// parse range_min and range_max
void parseRangeConfiguration(const Poco::Util::AbstractConfiguration & config, const std::string & structure_prefix);
};
}

View File

@ -12,7 +12,6 @@ namespace ErrorCodes
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INCONSISTENT_RESERVATIONS;
extern const int NO_RESERVATIONS_PROVIDED;
extern const int UNKNOWN_VOLUME_TYPE;
}
IVolume::IVolume(

View File

@ -106,8 +106,9 @@ private:
else if (buckets_field.getType() == Field::Types::UInt64)
num_buckets = checkBucketsRange(buckets_field.get<UInt64>());
else
throw Exception("Illegal type " + String(buckets_field.getTypeName()) + " of the second argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of the second argument of function {}",
buckets_field.getTypeName(), getName());
const auto & hash_col = arguments[0].column;
const IDataType * hash_type = arguments[0].type.get();

View File

@ -2585,8 +2585,9 @@ private:
}
else
{
throw Exception{"Conversion from " + std::string(getTypeName(from_type_index)) + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE,
"Conversion from {} to {} is not supported",
from_type_index, to_type->getName());
}
}
@ -2695,8 +2696,9 @@ private:
return nullable_column_wrapper(arguments, result_type, column_nullable, input_rows_count);
}
else
throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE,
"Conversion from {} to {} is not supported",
type_index, to_type->getName());
}
return result_column;

View File

@ -231,12 +231,10 @@ private:
key = col_fixed->getDataAt(offset + j).toString();
else if (const auto * col_str = checkAndGetColumn<ColumnString>(arg.key_column.get()))
key = col_str->getDataAt(offset + j).toString();
else
// should not happen
throw Exception(
"Expected String or FixedString, got " + std::string(getTypeName(arg.key_column->getDataType()))
+ " in " + getName(),
ErrorCodes::LOGICAL_ERROR);
else // should not happen
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected String or FixedString, got {} in {}",
arg.key_column->getDataType(), getName());
}
else
{

View File

@ -8,12 +8,8 @@
#include <functional>
#include <type_traits>
#include <common/EnumReflection.h>
namespace DB
{
/** Implements the ability to write and read data in/from WriteBuffer/ReadBuffer
* with the help of << and >> operators and also manipulators,
* providing a way of using, similar to iostreams.
@ -44,62 +40,23 @@ 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;
}
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, 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 WriteBuffer & operator<<(WriteBuffer & buf, const auto & x) { writeText(x, buf); return buf; }
inline 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; }
inline DoubleQuoteManipWriteBuffer operator<< (WriteBuffer & buf, DoubleQuoteManip) { return buf; }
inline BinaryManipWriteBuffer operator<< (WriteBuffer & buf, BinaryManip) { return buf; }
template <typename T> WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const T & x) { writeText(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const T & x) { writeQuoted(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const T & x) { writeText(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const T & x) { writeQuoted(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const T & x) { writeDoubleQuoted(x, buf.get()); return buf; }
template <typename T> WriteBuffer & operator<< (BinaryManipWriteBuffer buf, const T & x) { writeBinary(x, buf.get()); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const std::string_view & x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const StringRef & x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const String & x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, std::string_view x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, StringRef x) { writeEscapedString(x, buf); return buf; }
inline WriteBuffer & operator<< (EscapeManipWriteBuffer buf, const char * x) { writeEscapedString(x, strlen(x), buf); return buf; }
inline WriteBuffer & operator<< (QuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'\''>(x, x + strlen(x), buf.get()); return buf; }
inline WriteBuffer & operator<< (DoubleQuoteManipWriteBuffer buf, const char * x) { writeAnyQuotedString<'"'>(x, x + strlen(x), buf.get()); return buf; }
@ -128,5 +85,4 @@ template <typename T> ReadBuffer & operator>> (EscapeManipReadBuffer buf, T
template <typename T> ReadBuffer & operator>> (QuoteManipReadBuffer buf, T & x) { readQuoted(x, buf.get()); return buf; }
template <typename T> ReadBuffer & operator>> (DoubleQuoteManipReadBuffer buf, T & x) { readDoubleQuoted(x, buf.get()); return buf; }
template <typename T> ReadBuffer & operator>> (BinaryManipReadBuffer buf, T & x) { readBinary(x, buf.get()); return buf; }
}

View File

@ -13,6 +13,7 @@
#include <common/LocalDateTime.h>
#include <common/StringRef.h>
#include <common/arithmeticOverflow.h>
#include <common/Concepts.h>
#include <Core/Types.h>
#include <Core/DecimalFunctions.h>
@ -928,25 +929,23 @@ readBinaryBigEndian(T & x, ReadBuffer & buf) /// Assuming little endian archi
}
}
/// Generic methods to read value in text tab-separated format.
template <typename T>
inline std::enable_if_t<is_integer_v<T>, void>
readText(T & x, ReadBuffer & buf) { readIntText(x, buf); }
template <typename T>
inline std::enable_if_t<is_integer_v<T>, bool>
tryReadText(T & x, ReadBuffer & buf) { return tryReadIntText(x, buf); }
inline void readText(is_integer auto & x, ReadBuffer & buf)
{
if constexpr (std::same_as<decltype(x), bool &>)
readBoolText(x, buf);
else
readIntText(x, buf);
}
template <typename T>
inline std::enable_if_t<std::is_floating_point_v<T>, void>
readText(T & x, ReadBuffer & buf) { readFloatText(x, buf); }
inline bool tryReadText(is_integer auto & x, ReadBuffer & buf)
{
return tryReadIntText(x, buf);
}
template <typename T>
inline std::enable_if_t<std::is_floating_point_v<T>, bool>
tryReadText(T & x, ReadBuffer & buf) { return tryReadFloatText(x, buf); }
inline void readText(is_floating_point auto & x, ReadBuffer & buf) { readFloatText(x, buf); }
inline void readText(bool & x, ReadBuffer & buf) { readBoolText(x, buf); }
inline void readText(String & x, ReadBuffer & buf) { readEscapedString(x, buf); }
inline void readText(LocalDate & x, ReadBuffer & buf) { readDateText(x, buf); }
inline void readText(LocalDateTime & x, ReadBuffer & buf) { readDateTimeText(x, buf); }

View File

@ -1,32 +0,0 @@
#include <IO/ReadSettings.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_READ_METHOD;
}
const char * toString(ReadMethod read_method)
{
switch (read_method)
{
#define CASE_READ_METHOD(NAME) case ReadMethod::NAME: return #NAME;
FOR_EACH_READ_METHOD(CASE_READ_METHOD)
#undef CASE_READ_METHOD
}
__builtin_unreachable();
}
ReadMethod parseReadMethod(const std::string & name)
{
#define CASE_READ_METHOD(NAME) if (name == #NAME) return ReadMethod::NAME;
FOR_EACH_READ_METHOD(CASE_READ_METHOD)
#undef CASE_READ_METHOD
throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}'", name);
}
}

View File

@ -4,47 +4,45 @@
#include <string>
#include <Core/Defines.h>
namespace DB
{
#define FOR_EACH_READ_METHOD(M) \
/** Simple synchronous reads with 'read'. \
Can use direct IO after specified size. Can use prefetch by asking OS to perform readahead. */ \
M(read) \
\
/** Simple synchronous reads with 'pread'. \
In contrast to 'read', shares single file descriptor from multiple threads. \
Can use direct IO after specified size. Can use prefetch by asking OS to perform readahead. */ \
M(pread) \
\
/** Use mmap after specified size or simple synchronous reads with 'pread'. \
Can use prefetch by asking OS to perform readahead. */ \
M(mmap) \
\
/** Checks if data is in page cache with 'preadv2' on modern Linux kernels. \
If data is in page cache, read from the same thread. \
If not, offload IO to separate threadpool. \
Can do prefetch with double buffering. \
Can use specified priorities and limit the number of concurrent reads. */ \
M(pread_threadpool) \
\
/** It's using asynchronous reader with fake backend that in fact synchronous. \
Only used for testing purposes. */ \
M(pread_fake_async) \
enum class ReadMethod
{
#define DEFINE_READ_METHOD(NAME) NAME,
FOR_EACH_READ_METHOD(DEFINE_READ_METHOD)
#undef DEFINE_READ_METHOD
/**
* Simple synchronous reads with 'read'.
* Can use direct IO after specified size.
* Can use prefetch by asking OS to perform readahead.
*/
read,
/**
* Simple synchronous reads with 'pread'.
* In contrast to 'read', shares single file descriptor from multiple threads.
* Can use direct IO after specified size.
* Can use prefetch by asking OS to perform readahead.
*/
pread,
/**
* Use mmap after specified size or simple synchronous reads with 'pread'.
* Can use prefetch by asking OS to perform readahead.
*/
mmap,
/**
* Checks if data is in page cache with 'preadv2' on modern Linux kernels.
* If data is in page cache, read from the same thread.
* If not, offload IO to separate threadpool.
* Can do prefetch with double buffering.
* Can use specified priorities and limit the number of concurrent reads.
*/
pread_threadpool,
/// Use asynchronous reader with fake backend that in fact synchronous.
/// @attention Use only for testing purposes.
pread_fake_async
};
const char * toString(ReadMethod read_method);
ReadMethod parseReadMethod(const std::string & name);
class MMappedFileCache;
struct ReadSettings
@ -76,5 +74,4 @@ struct ReadSettings
return res;
}
};
}

View File

@ -5,6 +5,7 @@
#include <limits>
#include <algorithm>
#include <iterator>
#include <concepts>
#include <pcg-random/pcg_random.hpp>
@ -30,6 +31,8 @@
#include <IO/DoubleConverter.h>
#include <IO/WriteBufferFromString.h>
#include <common/EnumReflection.h>
#ifdef __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wunused-parameter"
@ -181,7 +184,8 @@ inline void writeString(const char * data, size_t size, WriteBuffer & buf)
buf.write(data, size);
}
inline void writeString(StringRef ref, WriteBuffer & buf)
// Otherwise StringRef and string_view overloads are ambiguous when passing string literal. Prefer std::string_view
void writeString(std::same_as<StringRef> auto ref, WriteBuffer & buf)
{
writeString(ref.data, ref.size, buf);
}
@ -878,24 +882,22 @@ inline void writeBinary(const LocalDateTime & x, WriteBuffer & buf) { writePODBi
inline void writeBinary(const UUID & x, WriteBuffer & buf) { writePODBinary(x, buf); }
/// Methods for outputting the value in text form for a tab-separated format.
template <typename T>
inline std::enable_if_t<is_integer_v<T>, void>
writeText(const T & x, WriteBuffer & buf) { writeIntText(x, buf); }
template <typename T>
inline std::enable_if_t<std::is_floating_point_v<T>, void>
writeText(const T & x, WriteBuffer & buf) { writeFloatText(x, buf); }
inline void writeText(is_integer auto x, WriteBuffer & buf)
{
if constexpr (std::same_as<decltype(x), bool>)
writeBoolText(x, buf);
else if constexpr (std::same_as<decltype(x), char>)
writeChar(x, buf);
else
writeIntText(x, buf);
}
inline void writeText(const String & x, WriteBuffer & buf) { writeString(x.c_str(), x.size(), buf); }
inline void writeText(const std::string_view & x, WriteBuffer & buf) { writeString(x.data(), x.size(), buf); }
inline void writeText(is_floating_point auto x, WriteBuffer & buf) { writeFloatText(x, buf); }
/// Implemented as template specialization (not function overload) to avoid preference over templates on arithmetic types above.
template <> inline void writeText<bool>(const bool & x, WriteBuffer & buf) { writeBoolText(x, buf); }
inline void writeText(is_enum auto x, WriteBuffer & buf) { writeText(magic_enum::enum_name(x), buf); }
/// unlike the method for std::string
/// assumes here that `x` is a null-terminated string.
inline void writeText(const char * x, WriteBuffer & buf) { writeCString(x, buf); }
inline void writeText(const char * x, size_t size, WriteBuffer & buf) { writeString(x, size, buf); }
inline void writeText(std::string_view x, WriteBuffer & buf) { writeString(x.data(), x.size(), buf); }
inline void writeText(const DayNum & x, WriteBuffer & buf) { writeDateText(LocalDate(x), buf); }
inline void writeText(const LocalDate & x, WriteBuffer & buf) { writeDateText(x, buf); }

View File

@ -185,9 +185,11 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co
/// If the function is not a tuple, treat it as a constant expression that returns tuple and extract it.
function_result = extractValueFromNode(elem, *tuple_type, context);
if (function_result.getType() != Field::Types::Tuple)
throw Exception("Invalid type of set. Expected tuple, got " + String(function_result.getTypeName()),
ErrorCodes::INCORRECT_ELEMENT_OF_SET);
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET,
"Invalid type of set. Expected tuple, got {}",
function_result.getTypeName());
tuple = &function_result.get<Tuple>();
}
@ -198,8 +200,9 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, Co
{
/// The literal must be tuple.
if (literal->value.getType() != Field::Types::Tuple)
throw Exception("Invalid type in set. Expected tuple, got "
+ String(literal->value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
throw Exception(ErrorCodes::INCORRECT_ELEMENT_OF_SET,
"Invalid type in set. Expected tuple, got {}",
literal->value.getTypeName());
tuple = &literal->value.get<Tuple>();
}

View File

@ -74,6 +74,7 @@
#include <Common/ShellCommand.h>
#include <Common/TraceCollector.h>
#include <common/logger_useful.h>
#include <common/EnumReflection.h>
#include <Common/RemoteHostFilter.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Interpreters/JIT/CompiledExpressionCache.h>
@ -118,6 +119,7 @@ namespace ErrorCodes
extern const int TABLE_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_READ_METHOD;
}
@ -2712,7 +2714,12 @@ ReadSettings Context::getReadSettings() const
{
ReadSettings res;
res.local_fs_method = parseReadMethod(settings.local_filesystem_read_method.value);
std::string_view read_method_str = settings.local_filesystem_read_method.value;
if (auto opt_method = magic_enum::enum_cast<ReadMethod>(read_method_str))
res.local_fs_method = *opt_method;
else
throw Exception(ErrorCodes::UNKNOWN_READ_METHOD, "Unknown read method '{}'", read_method_str);
res.local_fs_prefetch = settings.local_filesystem_read_prefetch;
res.remote_fs_prefetch = settings.remote_filesystem_read_prefetch;

View File

@ -149,8 +149,7 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data
if (!literal1 || !literal2)
return;
if (literal1->value.getTypeName() != std::string_view{"String"}
|| literal2->value.getTypeName() != std::string_view{"String"})
if (literal1->value.getTypeName() != "String" || literal2->value.getTypeName() != "String")
return;
changeIfArguments(function_node.arguments->children[1],
@ -166,8 +165,7 @@ void ConvertStringsToEnumMatcher::visit(ASTFunction & function_node, Data & data
if (!literal_to || !literal_other)
return;
if (literal_to->value.getTypeName() != std::string_view{"Array"}
|| literal_other->value.getTypeName() != std::string_view{"String"})
if (literal_to->value.getTypeName() != "Array" || literal_other->value.getTypeName() != "String")
return;
Array array_to = literal_to->value.get<Array>();

View File

@ -223,8 +223,9 @@ ExplainSettings<Settings> checkAndGetSettings(const ASTPtr & ast_settings)
"Supported settings: " + settings.getSettingsList(), ErrorCodes::UNKNOWN_SETTING);
if (change.value.getType() != Field::Types::UInt64)
throw Exception("Invalid type " + std::string(change.value.getTypeName()) + " for setting \"" + change.name +
"\" only boolean settings are supported", ErrorCodes::INVALID_SETTING_VALUE);
throw Exception(ErrorCodes::INVALID_SETTING_VALUE,
"Invalid type {} for setting \"{}\" only boolean settings are supported",
change.value.getTypeName(), change.name);
auto value = change.value.get<UInt64>();
if (value > 1)

View File

@ -346,8 +346,9 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast<const DataTypeAggregateFunction *>(&type))
{
if (src.getType() != Field::Types::AggregateFunctionState)
throw Exception(String("Cannot convert ") + src.getTypeName() + " to " + agg_func_type->getName(),
ErrorCodes::TYPE_MISMATCH);
throw Exception(ErrorCodes::TYPE_MISMATCH,
"Cannot convert {} to {}",
src.getTypeName(), agg_func_type->getName());
const auto & name = src.get<AggregateFunctionStateData>().name;
if (agg_func_type->getName() != name)

View File

@ -7,12 +7,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
void ASTSystemQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "SYSTEM ";

View File

@ -1,39 +1,24 @@
#include <Common/StringUtils/StringUtils.h>
#include <Parsers/CommonParsers.h>
#include <common/find_symbols.h>
#include <IO/Operators.h>
#include <string.h> /// strncmp, strncasecmp
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
bool ParserKeyword::parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected)
bool ParserKeyword::parseImpl(Pos & pos, [[maybe_unused]] ASTPtr & node, Expected & expected)
{
if (pos->type != TokenType::BareWord)
return false;
const char * current_word = s.begin();
const char * s_end = s.end();
const size_t s_length = s.length();
if (!s_length)
throw Exception("Logical error: keyword cannot be empty string", ErrorCodes::LOGICAL_ERROR);
while (true)
{
expected.add(pos, current_word);
if (pos->type != TokenType::BareWord)
return false;
const char * next_whitespace = find_first_symbols<' ', '\0'>(current_word, s_end);
size_t word_length = next_whitespace - current_word;
const char * const next_whitespace = find_first_symbols<' ', '\0'>(current_word, s.end());
const size_t word_length = next_whitespace - current_word;
if (word_length != pos->size())
return false;
@ -51,5 +36,4 @@ bool ParserKeyword::parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected)
return true;
}
}

View File

@ -16,10 +16,11 @@ private:
std::string_view s;
public:
constexpr ParserKeyword(std::string_view s_): s(s_) {}
//NOLINTNEXTLINE Want to be able to init ParserKeyword("literal")
constexpr ParserKeyword(std::string_view s_): s(s_) { assert(!s.empty()); }
protected:
const char * getName() const override { return s.data(); }
constexpr const char * getName() const override { return s.data(); }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};

View File

@ -70,9 +70,14 @@ bool ParserSystemQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Expected &
bool found = false;
// If query is executed on single replica, we want to parse input like FLUSH DISTRIBUTED
// If query is executed on cluster, we also want to parse serialized input like FLUSH_DISTRIBUTED
for (const auto & [entry, str] : magic_enum::enum_entries<Type>())
{
if (ParserKeyword(UnderscoreToSpace(str)).ignore(pos, expected))
String underscore_to_space(str);
std::replace(underscore_to_space.begin(), underscore_to_space.end(), '_', ' ');
if (ParserKeyword(underscore_to_space).ignore(pos, expected) || ParserKeyword(str).ignore(pos, expected))
{
res->type = entry;
found = true;

View File

@ -195,7 +195,7 @@ std::string getLexicalErrorMessage(
out << getErrorTokenDescription(last_token.type);
if (last_token.size())
{
out << ": '" << StringRef{last_token.begin, last_token.size()} << "'";
out << ": '" << std::string_view{last_token.begin, last_token.size()} << "'";
}
return out.str();

View File

@ -276,8 +276,9 @@ private:
info.type = std::make_shared<DataTypeMap>(nested_types);
}
else
throw Exception(String("Unexpected literal type ") + info.literal->value.getTypeName() + ". It's a bug",
ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Unexpected literal type {}",
info.literal->value.getTypeName());
/// Allow literal to be NULL, if result column has nullable type or if function never returns NULL
if (info.force_nullable && info.type->canBeInsideNullable())

View File

@ -41,9 +41,9 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, ContextPt
auto name_deps = getDependentViewsByColumn(context);
for (const auto & command : commands)
{
if (command.type != AlterCommand::Type::ADD_COLUMN
if (command.type != AlterCommand::Type::ADD_COLUMN
&& command.type != AlterCommand::Type::MODIFY_COLUMN
&& command.type != AlterCommand::Type::DROP_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());

View File

@ -42,7 +42,7 @@ Alter ALTER TABLE sqllt.table DROP COLUMN the_new_col;
Alter ALTER TABLE sqllt.table UPDATE i = i + 1 WHERE 1;
Alter ALTER TABLE sqllt.table DELETE WHERE i > 65535;
Select -- not done, seems to hard, so I\'ve skipped queries of ALTER-X, where X is:\n-- PARTITION\n-- ORDER BY\n-- SAMPLE BY\n-- INDEX\n-- CONSTRAINT\n-- TTL\n-- USER\n-- QUOTA\n-- ROLE\n-- ROW POLICY\n-- SETTINGS PROFILE\n\nSELECT \'SYSTEM queries\';
System SYSTEM FLUSH LOGS;
System SYSTEM FLUSH LOGS
System SYSTEM STOP MERGES sqllt.table
System SYSTEM START MERGES sqllt.table
System SYSTEM STOP TTL MERGES sqllt.table

View File

@ -33,7 +33,7 @@ def syntax_error():
return (62, "Exception: Syntax error")
def groups_frame_error():
return (48, "Exception: Window frame 'GROUPS' is not implemented")
return (48, "Exception: Window frame 'Groups' is not implemented")
def getuid():
if current().subtype == TestSubType.Example: