This commit is contained in:
Alexey Arno 2016-05-04 02:19:14 +03:00
parent 2b1550105c
commit 286baa8611
34 changed files with 30013 additions and 1081 deletions

View File

@ -28,7 +28,9 @@ add_library (dbms
include/DB/Functions/FunctionsString.h
include/DB/Functions/FunctionsRound.h
include/DB/Functions/FunctionsTransform.h
include/DB/Functions/Conditional/CondException.h
include/DB/Functions/Conditional/common.h
include/DB/Functions/Conditional/getArrayType.h
include/DB/Functions/Conditional/ArgsInfo.h
include/DB/Functions/Conditional/CondSource.h
include/DB/Functions/Conditional/NumericPerformer.h
@ -51,7 +53,8 @@ add_library (dbms
include/DB/Functions/IFunction.h
include/DB/Functions/FunctionsFormatting.h
include/DB/Functions/NumberTraits.h
include/DB/Functions/DataTypeFromFieldTypeOrError.h
include/DB/Functions/DataTypeTraits.h
include/DB/Functions/EnrichedDataTypePtr.h
include/DB/TableFunctions/TableFunctionRemote.h
include/DB/TableFunctions/TableFunctionFactory.h
include/DB/TableFunctions/TableFunctionMerge.h
@ -114,6 +117,7 @@ add_library (dbms
include/DB/Parsers/IParser.h
include/DB/Parsers/ASTSampleRatio.h
include/DB/Parsers/ParserSampleRatio.h
include/DB/Parsers/ParserCase.h
include/DB/AggregateFunctions/AggregateFunctionMerge.h
include/DB/AggregateFunctions/AggregateFunctionUniqUpTo.h
include/DB/AggregateFunctions/AggregateFunctionIf.h
@ -761,6 +765,7 @@ add_library (dbms
src/Parsers/ParserAlterQuery.cpp
src/Parsers/ParserCheckQuery.cpp
src/Parsers/ParserSampleRatio.cpp
src/Parsers/ParserCase.cpp
src/Parsers/formatAST.cpp
src/Parsers/parseQuery.cpp
src/Parsers/queryToString.cpp
@ -827,6 +832,7 @@ add_library (dbms
src/Functions/FunctionsMath.cpp
src/Functions/FunctionsMiscellaneous.cpp
src/Functions/FunctionsTransform.cpp
src/Functions/Conditional/getArrayType.cpp
src/Functions/Conditional/ArgsInfo.cpp
src/Functions/Conditional/CondSource.cpp
src/Functions/Conditional/NumericPerformer.cpp

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <DB/Functions/EnrichedDataTypePtr.h>
namespace DB
@ -12,6 +13,8 @@ using Poco::SharedPtr;
class DataTypeArray final : public IDataType
{
private:
/// Расширенный тип элементов массивов.
DataTypeTraits::EnrichedDataTypePtr enriched_nested;
/// Тип элементов массивов.
DataTypePtr nested;
/// Тип смещений.
@ -19,6 +22,7 @@ private:
public:
DataTypeArray(DataTypePtr nested_);
DataTypeArray(DataTypeTraits::EnrichedDataTypePtr enriched_nested_);
std::string getName() const override
{
@ -27,7 +31,7 @@ public:
DataTypePtr clone() const override
{
return new DataTypeArray(nested);
return new DataTypeArray(enriched_nested);
}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
@ -82,6 +86,7 @@ public:
}
const DataTypePtr & getNestedType() const { return nested; }
const DataTypeTraits::EnrichedDataTypePtr & getEnrichedNestedType() const { return enriched_nested; }
const DataTypePtr & getOffsetsType() const { return offsets; }
};

View File

@ -40,9 +40,18 @@ DEFINE_DATA_TYPE_NUMBER_FIXED(Float64);
/// The following type is not a real column data type. It is used in the multiIf
/// function implementation for argument type checking.
class DataTypeVoid : public IDataTypeNumberFixed<void, void>
{
public:
DataTypeVoid() = default;
std::string getName() const override { return "void"; }
DataTypePtr clone() const override { return new DataTypeVoid; }
};
template <> struct DataTypeFromFieldType<void>
{
typedef void Type;
typedef DataTypeVoid Type;
};
}

View File

@ -108,6 +108,27 @@ public:
}
};
template <>
class IDataTypeNumber<void> : public IDataType
{
public:
using FieldType = void;
bool isNumeric() const override { return true; }
bool behavesAsNumber() const override { return true; }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override {}
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const override {}
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override {}
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override {}
size_t getSizeOfField() const override { return 0; }
Field getDefault() const override { return {}; }
};
template <typename FType> inline void IDataTypeNumber<FType>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
serializeText(column, row_num, ostr);

View File

@ -83,5 +83,19 @@ public:
}
};
template <>
class IDataTypeNumberFixed<void, void> : public IDataTypeNumber<void>
{
public:
void serializeBinary(const Field & field, WriteBuffer & ostr) const override {}
void deserializeBinary(Field & field, ReadBuffer & istr) const override {}
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override {}
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override {}
void serializeBinary(const IColumn & column, WriteBuffer & ostr, size_t offset = 0, size_t limit = 0) const override {}
void deserializeBinary(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override {}
ColumnPtr createColumn() const override { return {}; }
ColumnPtr createConstColumn(size_t size, const Field & field) const override { return {}; }
};
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/DataTypes/DataTypeArray.h>
@ -13,7 +14,6 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -317,10 +317,6 @@ public:
}
}
private:
template <typename TType>
using ConcreteArraySourceCreator = ArraySourceCreator<TResult, TType>;
private:
/// Create accessors for condition values.
static CondSources createConds(const Block & block, const ColumnNumbers & args)
@ -342,7 +338,16 @@ private:
for (const auto & br : branches)
{
if (!NumericTypeDispatcher<ConcreteArraySourceCreator>::apply(sources, block, args, br))
if (! (ArraySourceCreator<TResult, UInt8>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, UInt16>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, UInt32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, UInt64>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Int8>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Int16>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Int32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Int64>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Float32>::execute(sources, block, args, br)
|| ArraySourceCreator<TResult, Float64>::execute(sources, block, args, br)))
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
}
@ -393,8 +398,7 @@ class ArrayEvaluator<NumberTraits::Error>
public:
static void perform(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error: one or more arguments of function "
"multiIf have invalid types", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw CondException{CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES};
}
};

View File

@ -0,0 +1,48 @@
#pragma once
#include <DB/Common/Exception.h>
namespace DB
{
namespace Conditional
{
enum class CondErrorCodes
{
TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE,
TYPE_DEDUCER_UPSCALING_ERROR,
NUMERIC_PERFORMER_ILLEGAL_COLUMN,
COND_SOURCE_ILLEGAL_COLUMN,
NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT,
ARRAY_EVALUATOR_INVALID_TYPES
};
/// Since the building blocks of the multiIf function may be called
/// in various contexts, their error management must be achieved in
/// a context-free manner. Hence the need for the following class.
class CondException : public DB::Exception
{
public:
CondException(CondErrorCodes code_, const std::string & msg1_ = "", const std::string & msg2_ = "")
: code{code_}, msg1{msg1_}, msg2{msg2_}
{
}
const char * name() const throw() override { return "DB::Conditional::Exception"; }
const char * className() const throw() override { return "DB::Conditional::Exception"; }
CondException * clone() const override { return new CondException{*this}; }
void rethrow() const override { throw *this; }
CondErrorCodes getCode() const { return code; }
std::string getMsg1() const { return msg1; }
std::string getMsg2() const { return msg2; }
private:
CondErrorCodes code;
std::string msg1;
std::string msg2;
};
}
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/Functions/NumberTraits.h>
@ -161,10 +162,6 @@ public:
}
}
private:
template <typename TType>
using ConcreteNumericSourceCreator = NumericSourceCreator<TResult, TType>;
private:
/// Create the result column.
static PaddedPODArray<TResult> & createSink(Block & block, size_t result, size_t size)
@ -200,9 +197,17 @@ private:
{
NumericSourcePtr<TResult> source;
if (!NumericTypeDispatcher<ConcreteNumericSourceCreator>::apply(source, block, args, br))
throw Exception{"Illegal type of argument " + toString(br.index) + " of function multiIf",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (! (NumericSourceCreator<TResult, UInt8>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, UInt16>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, UInt32>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, UInt64>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Int8>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Int16>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Int32>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Int64>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Float32>::execute(source, block, args, br)
|| NumericSourceCreator<TResult, Float64>::execute(source, block, args, br)))
throw CondException{CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT, toString(br.index)};
sources.push_back(std::move(source));
}

View File

@ -1,87 +1,11 @@
#pragma once
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
namespace Conditional
{
/// Execute a given parametrized predicate for each type from a given list of
/// types until it returns true for one of these types.
template <template <typename> class Predicate, typename TType, typename... RTypeList>
struct Disjunction final
{
template <typename... Args>
static bool apply(Args &&... args)
{
return Predicate<TType>::execute(std::forward<Args>(args)...)
|| Disjunction<Predicate, RTypeList...>::apply(std::forward<Args>(args)...);
}
};
template <template <typename> class Predicate, typename TType>
struct Disjunction<Predicate, TType>
{
template <typename... Args>
static bool apply(Args &&... args)
{
return Predicate<TType>::execute(std::forward<Args>(args)...);
}
};
/// Common code for NumericTypeDispatcher and DataTypeDispatcher.
/// See comments below.
template <template <typename> class Predicate, bool isNumeric>
struct TypeDispatcher final
{
private:
template <typename TType, bool isNumeric2>
struct ActualType;
template <typename TType>
struct ActualType<TType, true>
{
using Type = TType;
};
template <typename TType>
struct ActualType<TType, false>
{
using Type = typename DataTypeFromFieldType<TType>::Type;
};
public:
template <typename... Args>
static bool apply(Args&&... args)
{
return Disjunction<
Predicate,
typename ActualType<UInt8, isNumeric>::Type,
typename ActualType<UInt16, isNumeric>::Type,
typename ActualType<UInt32, isNumeric>::Type,
typename ActualType<UInt64, isNumeric>::Type,
typename ActualType<Int8, isNumeric>::Type,
typename ActualType<Int16, isNumeric>::Type,
typename ActualType<Int32, isNumeric>::Type,
typename ActualType<Int64, isNumeric>::Type,
typename ActualType<Float32, isNumeric>::Type,
typename ActualType<Float64, isNumeric>::Type
>::apply(std::forward<Args>(args)...);
}
};
/// Execute a given parametrized predicate for each numeric type
/// until it returns true for such a numeric type.
template <template <typename> class Predicate>
using NumericTypeDispatcher = TypeDispatcher<Predicate, true>;
/// Execute a given parametrized predicate for each data type
/// until it returns true for such a data type.
template <template <typename> class Predicate>
using DataTypeDispatcher = TypeDispatcher<Predicate, false>;
/// When performing a multiIf for numeric arguments, the following
/// structure is used to collect all the information needed on
/// the branches (1 or more then branches + 1 else branch) for

View File

@ -0,0 +1,17 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <DB/Functions/EnrichedDataTypePtr.h>
namespace DB
{
namespace Conditional
{
/// Determine the least common type of the elements of an array.
DataTypeTraits::EnrichedDataTypePtr getArrayType(const DataTypes & args);
}
}

View File

@ -1,26 +0,0 @@
#pragma once
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
}

View File

@ -0,0 +1,137 @@
#pragma once
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/EnrichedDataTypePtr.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
namespace DB
{
namespace DataTypeTraits
{
template <typename T>
struct DataTypeFromFieldTypeOrError
{
static DataTypePtr getDataType()
{
return new typename DataTypeFromFieldType<T>::Type;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
{
static DataTypePtr getDataType()
{
return nullptr;
}
};
/// Convert an enriched data type into an enriched numberic type.
template <typename T>
struct ToEnrichedNumericType
{
private:
using Type0 = typename std::tuple_element<0, T>::type;
using Type1 = typename std::tuple_element<1, T>::type;
public:
using Type = std::tuple<
typename Type0::FieldType,
typename Type1::FieldType
>;
};
/// Convert an enriched numeric type into an enriched data type.
template <typename T>
struct ToEnrichedDataType
{
private:
using Type0 = typename std::tuple_element<0, T>::type;
using Type1 = typename std::tuple_element<1, T>::type;
public:
using Type = std::tuple<
typename DataTypeFromFieldType<Type0>::Type,
typename DataTypeFromFieldType<Type1>::Type
>;
};
/// Convert an enriched numeric type into an enriched data type.
/// Error case.
template <>
struct ToEnrichedDataType<NumberTraits::Error>
{
using Type = NumberTraits::Error;
};
template <typename TEnrichedType, bool isNumeric>
struct ToEnrichedDataTypeObject;
/// Convert an enriched numeric type into an enriched data type object.
template <typename TEnrichedType>
struct ToEnrichedDataTypeObject<TEnrichedType, true>
{
static EnrichedDataTypePtr execute()
{
using Type0 = typename std::tuple_element<0, TEnrichedType>::type;
using DataType0 = typename DataTypeFromFieldType<Type0>::Type;
using Type1 = typename std::tuple_element<1, TEnrichedType>::type;
using DataType1 = typename DataTypeFromFieldType<Type1>::Type;
return std::make_pair(new DataType0{}, new DataType1{});
}
};
/// Convert an enriched data type into an enriched data type object.
template <typename TEnrichedType>
struct ToEnrichedDataTypeObject<TEnrichedType, false>
{
static EnrichedDataTypePtr execute()
{
using DataType0 = typename std::tuple_element<0, TEnrichedType>::type;
using DataType1 = typename std::tuple_element<1, TEnrichedType>::type;
return std::make_pair(new DataType0{}, new DataType1{});
}
};
/// Convert an enriched numeric type into an enriched data type object.
/// Error case.
template <>
struct ToEnrichedDataTypeObject<NumberTraits::Error, true>
{
static EnrichedDataTypePtr execute()
{
return std::make_pair(nullptr, nullptr);
}
};
/// Convert an enriched data type into an enriched data type object.
/// Error case.
template <>
struct ToEnrichedDataTypeObject<NumberTraits::Error, false>
{
static EnrichedDataTypePtr execute()
{
return std::make_pair(nullptr, nullptr);
}
};
/// Compute the product of an enriched data type with an ordinary data type.
template <typename T1, typename T2>
struct DataTypeProduct
{
using Type = typename ToEnrichedDataType<
typename NumberTraits::TypeProduct<
typename ToEnrichedNumericType<T1>::Type,
typename NumberTraits::EmbedType<typename T2::FieldType>::Type
>::Type
>::Type;
};
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <utility>
namespace DB
{
namespace DataTypeTraits
{
/// This type is declared in a separate header in order to increase
/// compilation speed.
using EnrichedDataTypePtr = std::pair<DataTypePtr, DataTypePtr>;
}
}

View File

@ -16,8 +16,8 @@
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Common/HashTable/ClearableHashMap.h>
#include <DB/Interpreters/AggregationCommon.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionsConditional.h>
#include <DB/Functions/Conditional/getArrayType.h>
#include <DB/AggregateFunctions/IAggregateFunction.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/Parsers/ExpressionListParsers.h>
@ -76,45 +76,7 @@ private:
/// Получить имя функции.
String getName() const override
{
return name;
}
template <typename T0, typename T1>
bool checkRightType(DataTypePtr left, DataTypePtr right, DataTypePtr & type_res) const
{
if (typeid_cast<const T1 *>(&*right))
{
typedef typename NumberTraits::ResultOfIf<typename T0::FieldType, typename T1::FieldType>::Type ResultType;
type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Arguments of function " + getName() + " are not upscalable to a common type without loss of precision.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return true;
}
return false;
}
template <typename T0>
bool checkLeftType(DataTypePtr left, DataTypePtr right, DataTypePtr & type_res) const
{
if (typeid_cast<const T0 *>(&*left))
{
if ( checkRightType<T0, DataTypeUInt8>(left, right, type_res)
|| checkRightType<T0, DataTypeUInt16>(left, right, type_res)
|| checkRightType<T0, DataTypeUInt32>(left, right, type_res)
|| checkRightType<T0, DataTypeUInt64>(left, right, type_res)
|| checkRightType<T0, DataTypeInt8>(left, right, type_res)
|| checkRightType<T0, DataTypeInt16>(left, right, type_res)
|| checkRightType<T0, DataTypeInt32>(left, right, type_res)
|| checkRightType<T0, DataTypeInt64>(left, right, type_res)
|| checkRightType<T0, DataTypeFloat32>(left, right, type_res)
|| checkRightType<T0, DataTypeFloat64>(left, right, type_res))
return true;
else
throw Exception("Illegal type " + right->getName() + " as argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return false;
return is_case_mode ? "CASE" : name;
}
template <typename T0, typename T1>
@ -143,26 +105,14 @@ private:
|| tryAddField<DataTypeFloat64, DataTypeFloat64>(type_res, f, arr) )
return true;
else
throw Exception("Illegal result type " + type_res->getName() + " of function " + getName(),
ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getLeastCommonType(DataTypePtr left, DataTypePtr right) const
{
DataTypePtr type_res;
if (!( checkLeftType<DataTypeUInt8>(left, right, type_res)
|| checkLeftType<DataTypeUInt16>(left, right, type_res)
|| checkLeftType<DataTypeUInt32>(left, right, type_res)
|| checkLeftType<DataTypeUInt64>(left, right, type_res)
|| checkLeftType<DataTypeInt8>(left, right, type_res)
|| checkLeftType<DataTypeInt16>(left, right, type_res)
|| checkLeftType<DataTypeInt32>(left, right, type_res)
|| checkLeftType<DataTypeInt64>(left, right, type_res)
|| checkLeftType<DataTypeFloat32>(left, right, type_res)
|| checkLeftType<DataTypeFloat64>(left, right, type_res)))
throw Exception("Internal error: unexpected type " + left->getName() + " as argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
{
if (is_case_mode)
throw Exception{"Illegal type encountered while processing the CASE construction.",
ErrorCodes::LOGICAL_ERROR};
else
throw Exception{"Illegal result type " + type_res->getName() + " of function " + getName(),
ErrorCodes::LOGICAL_ERROR};
}
}
static const DataTypePtr & getScalarType(const DataTypePtr & type)
@ -175,30 +125,92 @@ private:
return getScalarType(array->getNestedType());
}
DataTypeTraits::EnrichedDataTypePtr getLeastCommonType(const DataTypes & arguments) const
{
DataTypeTraits::EnrichedDataTypePtr result_type;
try
{
result_type = Conditional::getArrayType(arguments);
}
catch (const Conditional::CondException & ex)
{
/// Translate a context-free error into a contextual error.
if (is_case_mode)
{
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" in CASE construction", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception{"THEN/ELSE clause parameters in CASE construction are not upscalable to a "
"common type without loss of precision: " + ex.getMsg1(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"An unexpected error has occurred in CASE expression",
ErrorCodes::LOGICAL_ERROR};
}
else
{
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" in array", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception("Arguments of function " + getName() + " are not upscalable "
"to a common type without loss of precision.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception{"An unexpected error has occurred in function " + getName(),
ErrorCodes::LOGICAL_ERROR};
}
}
return result_type;
}
public:
void setCaseMode()
{
is_case_mode = true;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
if (arguments.empty())
throw Exception("Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
{
if (is_case_mode)
throw Exception{"Either WHEN clauses or THEN clauses are missing "
"in the CASE construction.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
else
throw Exception{"Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
DataTypePtr result_type = arguments[0];
if (result_type->behavesAsNumber())
{
/// Если тип числовой, пробуем выделить наименьший общий тип
for (size_t i = 1, size = arguments.size(); i < size; ++i)
result_type = getLeastCommonType(result_type, arguments[i]);
auto enriched_result_type = getLeastCommonType(arguments);
return new DataTypeArray{enriched_result_type};
}
else
{
/// Иначе все аргументы должны быть одинаковыми
for (size_t i = 1, size = arguments.size(); i < size; ++i)
{
if (arguments[i]->getName() != arguments[0]->getName())
throw Exception("Arguments for function array must have same type or behave as number.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
{
if (is_case_mode)
throw Exception{"Found type discrepancy in either WHEN "
"clauses or THEN clauses of the CASE construction",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Arguments for function array must have same type or behave as number.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
}
return new DataTypeArray(result_type);
return new DataTypeArray{result_type};
}
}
/// Выполнить функцию над блоком.
@ -214,11 +226,18 @@ public:
const auto first_arg = block.getByPosition(arguments[0]);
DataTypePtr result_type = first_arg.type;
DataTypeTraits::EnrichedDataTypePtr enriched_result_type;
if (result_type->behavesAsNumber())
{
/// Если тип числовой, вычисляем наименьший общий тип
for (size_t i = 1, size = arguments.size(); i < size; ++i)
result_type = getLeastCommonType(result_type, block.getByPosition(arguments[i]).type);
DataTypes types;
types.reserve(arguments.size());
for (const auto & argument : arguments)
types.push_back(block.getByPosition(argument).type);
enriched_result_type = getLeastCommonType(types);
result_type = enriched_result_type.first;
}
if (is_const)
@ -259,6 +278,9 @@ public:
block.getByPosition(result).column = out_ptr;
}
}
private:
bool is_case_mode = false;
};

View File

@ -11,9 +11,10 @@
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/DataTypeFromFieldTypeOrError.h>
#include <DB/Functions/DataTypeTraits.h>
/// The following includes are needed for the function multiIf.
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/NumericPerformer.h>
@ -829,7 +830,7 @@ private:
if (typeid_cast<const T1 *>(&*arguments[2]))
{
typedef typename NumberTraits::ResultOfIf<typename T0::FieldType, typename T1::FieldType>::Type ResultType;
type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
type_res = DataTypeTraits::DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Arguments 2 and 3 of function " + getName() + " are not upscalable to a common type without loss of precision: "
+ arguments[1]->getName() + " and " + arguments[2]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -1400,60 +1401,89 @@ public:
static constexpr auto name = "multiIf";
static IFunction * create(const Context & context) { return new FunctionMultiIf; }
private:
bool performTrivialCase(Block & block, const ColumnNumbers & args, size_t result)
{
size_t else_arg = Conditional::elseArg(args);
auto first_type_name = block.getByPosition(args[Conditional::firstThen()]).type->getName();
for (size_t i = Conditional::secondThen(); i < else_arg; i = Conditional::nextThen(i))
{
if (block.getByPosition(args[i]).type->getName() != first_type_name)
return false;
}
if (block.getByPosition(args.back()).type->getName() != first_type_name)
return false;
auto & res_col = block.getByPosition(result).column;
for (size_t i = Conditional::firstCond(); i < else_arg; i = Conditional::nextCond(i))
{
auto cond_const_col = typeid_cast<const ColumnConst<UInt8> *>(&*block.getByPosition(args[i]).column);
if (!cond_const_col)
return false;
bool has_triggered_cond = cond_const_col->getData();
if (has_triggered_cond)
{
res_col = block.getByPosition(args[Conditional::thenFromCond(i)]).column;
return true;
}
}
res_col = block.getByPosition(args[else_arg]).column;
return true;
}
public:
String getName() const override
{
return name;
return is_case_mode ? "CASE" : name;
}
void setCaseMode()
{
is_case_mode = true;
}
DataTypePtr getReturnType(const DataTypes & args) const override
{
DataTypePtr data_type;
try
{
data_type = getReturnTypeImpl(args);
}
catch (const Conditional::CondException & ex)
{
rethrowContextually(ex);
}
return data_type;
}
void execute(Block & block, const ColumnNumbers & args, size_t result) override
{
try
{
if (performTrivialCase(block, args, result))
return;
if (Conditional::NumericPerformer::perform(block, args, result))
return;
if (Conditional::StringEvaluator::perform(block, args, result))
return;
if (Conditional::StringArrayEvaluator::perform(block, args, result))
return;
if (is_case_mode)
throw Exception{"Some THEN/ELSE clauses in CASE construction have "
"illegal or incompatible types", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"One or more branch (then, else) columns of function "
+ getName() + " have illegal or incompatible types",
ErrorCodes::ILLEGAL_COLUMN};
}
catch (const Conditional::CondException & ex)
{
rethrowContextually(ex);
}
}
private:
DataTypePtr getReturnTypeImpl(const DataTypes & args) const
{
if (!Conditional::hasValidArgCount(args))
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
{
if (is_case_mode)
throw Exception{"Some mandatory parameters are missing in the CASE "
"construction", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
else
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
/// Check conditions types.
for (size_t i = Conditional::firstCond(); i < Conditional::elseArg(args); i = Conditional::nextCond(i))
{
if (!typeid_cast<const DataTypeUInt8 *>(&*args[i]))
throw Exception{"Illegal type of argument " + toString(i) + " (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
{
if (is_case_mode)
throw Exception{"In CASE construction, illegal type of WHEN clause "
+ toString(i / 2) + ". Must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Illegal type of argument " + toString(i) + " (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
}
if (Conditional::hasArithmeticBranches(args))
@ -1489,9 +1519,16 @@ public:
if (Conditional::hasFixedStrings(args))
{
if (!Conditional::hasFixedStringsOfIdenticalLength(args))
throw Exception{"Branch (then, else) arguments of function " + getName() +
" have FixedString type and different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
{
if (is_case_mode)
throw Exception{"THEN/ELSE clauses in CASE construction "
"have FixedString type and different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Branch (then, else) arguments of function " + getName() +
" have FixedString type and different sizes",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const IDataType * data = args[Conditional::firstThen()].get();
const auto * fixed_str = typeid_cast<const DataTypeFixedString *>(data);
@ -1504,33 +1541,138 @@ public:
else if (Conditional::hasStrings(args))
return new DataTypeString;
else
throw Exception{
"Incompatible branch (then, else) arguments for function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
{
if (is_case_mode)
throw Exception{"THEN/ELSE clauses in CASE construction "
"have incompatible arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{
"Incompatible branch (then, else) arguments for function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
}
else
return args[Conditional::firstThen()];
}
void execute(Block & block, const ColumnNumbers & args, size_t result) override
bool performTrivialCase(Block & block, const ColumnNumbers & args, size_t result)
{
if (performTrivialCase(block, args, result))
return;
size_t else_arg = Conditional::elseArg(args);
auto first_type_name = block.getByPosition(args[Conditional::firstThen()]).type->getName();
if (Conditional::NumericPerformer::perform(block, args, result))
return;
for (size_t i = Conditional::secondThen(); i < else_arg; i = Conditional::nextThen(i))
{
if (block.getByPosition(args[i]).type->getName() != first_type_name)
return false;
}
if (Conditional::StringEvaluator::perform(block, args, result))
return;
if (block.getByPosition(args.back()).type->getName() != first_type_name)
return false;
if (Conditional::StringArrayEvaluator::perform(block, args, result))
return;
auto & res_col = block.getByPosition(result).column;
throw Exception{"One or more branch (then, else) columns of function have"
" illegal or incompatible types " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
for (size_t i = Conditional::firstCond(); i < else_arg; i = Conditional::nextCond(i))
{
auto cond_const_col = typeid_cast<const ColumnConst<UInt8> *>(&*block.getByPosition(args[i]).column);
if (!cond_const_col)
return false;
bool has_triggered_cond = cond_const_col->getData();
if (has_triggered_cond)
{
res_col = block.getByPosition(args[Conditional::thenFromCond(i)]).column;
return true;
}
}
res_col = block.getByPosition(args[else_arg]).column;
return true;
}
/// Translate a context-free error into a contextual error.
void rethrowContextually(const Conditional::CondException & ex) const
{
if (is_case_mode)
{
/// CASE construction context.
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" in CASE construction", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception{"THEN/ELSE clause parameters in CASE construction are not upscalable to a "
"common type without loss of precision: " + ex.getMsg1(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN)
{
size_t i = std::stoul(ex.getMsg1());
if ((i % 2) == 1)
throw Exception{"Illegal THEN clause " + toString(1 + (i - 1) / 2)
+ " in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
}
else if (ex.getCode() == Conditional::CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN)
{
size_t i = std::stoul(ex.getMsg2());
if ((i % 2) == 1)
throw Exception{"Illegal column " + ex.getMsg1() + " of THEN clause "
+ toString(1 + (i - 1) / 2) + " in CASE construction."
"Must be ColumnUInt8 or ColumnConstUInt8", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + ex.getMsg1() + " of ELSE clause "
" in CASE construction. Must be ColumnUInt8 or ColumnConstUInt8",
ErrorCodes::ILLEGAL_COLUMN};
}
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT)
{
size_t i = std::stoul(ex.getMsg1());
if ((i % 2) == 1)
throw Exception{"Illegal type of THEN clause " + toString(1 + (i - 1) / 2)
+ " in CASE construction", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Illegal type of ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
else if (ex.getCode() == Conditional::CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES)
throw Exception{"Internal logic error: one or more THEN/ELSE clauses of "
"CASE construction have invalid types", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"An unexpected error has occurred in CASE construction",
ErrorCodes::LOGICAL_ERROR};
}
else
{
/// multiIf function context.
if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE)
throw Exception{"Illegal type of column " + ex.getMsg1() +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR)
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + ex.getMsg1(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN)
throw Exception{"Illegal argument " + ex.getMsg1() + " of function multiIf",
ErrorCodes::ILLEGAL_COLUMN};
else if (ex.getCode() == Conditional::CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN)
throw Exception{"Illegal column " + ex.getMsg1() + " of argument "
+ ex.getMsg2() + " of function multiIf"
"Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN};
else if (ex.getCode() == Conditional::CondErrorCodes::NUMERIC_EVALUATOR_ILLEGAL_ARGUMENT)
throw Exception{"Illegal type of argument " + ex.getMsg1() + " of function multiIf",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else if (ex.getCode() == Conditional::CondErrorCodes::ARRAY_EVALUATOR_INVALID_TYPES)
throw Exception{"Internal logic error: one or more arguments of function "
"multiIf have invalid types", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"An unexpected error has occurred while performing multiIf",
ErrorCodes::LOGICAL_ERROR};
}
}
private:
bool is_case_mode = false;
};
}

View File

@ -13,6 +13,7 @@
#include <DB/Core/StringRef.h>
#include <DB/Common/HashTable/HashMap.h>
#include <DB/Functions/IFunction.h>
#include <DB/Functions/EnrichedDataTypePtr.h>
namespace DB
@ -21,7 +22,7 @@ namespace DB
/** transform(x, from_array, to_array[, default]) - преобразовать x согласно переданному явным образом соответствию.
*/
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2);
DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeTraits::EnrichedDataTypePtr & type1, const IDataType & type2);
/** transform(x, [from...], [to...], default)
* - преобразует значения согласно явно указанному отображению.
@ -49,52 +50,101 @@ public:
static constexpr auto name = "transform";
static IFunction * create(const Context &) { return new FunctionTransform; }
String getName() const override { return name; }
String getName() const override
{
return is_case_mode ? "CASE" : name;
}
void setCaseMode()
{
is_case_mode = true;
}
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
const auto args_size = arguments.size();
if (args_size != 3 && args_size != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
{
if (is_case_mode)
throw Exception{"Some mandatory parameters are missing in CASE construction",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
else
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(args_size) + ", should be 3 or 4",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
}
const IDataType * type_x = arguments[0].get();
if (!type_x->isNumeric() && !typeid_cast<const DataTypeString *>(type_x))
throw Exception("Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"Unsupported type " + type_x->getName()
+ " of parameter of the CASE clause"
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Unsupported type " + type_x->getName()
+ " of first argument of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const DataTypeArray * type_arr_from = typeid_cast<const DataTypeArray *>(arguments[1].get());
if (!type_arr_from)
throw Exception("Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"An internal error has been encountered while checking WHEN clauses "
"in CASE construction", ErrorCodes::LOGICAL_ERROR};
else
throw Exception{"Second argument of function " + getName()
+ ", must be array of source values to transform from.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const auto type_arr_from_nested = type_arr_from->getNestedType();
if ((type_x->isNumeric() != type_arr_from_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_from_nested.get())))
throw Exception("First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"The CASE clause and WHEN clauses in CASE construction "
"must have compatible types: both numeric or both strings", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"First argument and elements of array of second argument of function " + getName()
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const DataTypeArray * type_arr_to = typeid_cast<const DataTypeArray *>(arguments[2].get());
if (!type_arr_to)
throw Exception("Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"An internal error has been encountered while checking THEN clauses "
"in CASE construction", ErrorCodes::LOGICAL_ERROR};
else
throw Exception{"Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
const auto type_arr_to_nested = type_arr_to->getNestedType();
const auto enriched_type_arr_to_nested = type_arr_to->getEnrichedNestedType();
const auto & type_arr_to_nested = enriched_type_arr_to_nested.first;
if (args_size == 3)
{
if ((type_x->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_x) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"CASE constructions must satisfy either of the following two conditions: "
"1. CASE clause and THEN clauses have common type T; "
"ELSE clause and WHEN clauses have common type U; "
"2. The CASE construction has no ELSE clause; "
"All the clauses have common type T", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Function " + getName()
+ " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return type_x->clone();
}
@ -103,20 +153,37 @@ public:
const IDataType * type_default = arguments[3].get();
if (!type_default->isNumeric() && !typeid_cast<const DataTypeString *>(type_default))
throw Exception("Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"Unsupported type " + type_default->getName()
+ " of the ELSE clause of the CASE expression,"
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Unsupported type " + type_default->getName()
+ " of fourth argument (default value) of function " + getName()
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if ((type_default->isNumeric() != type_arr_to_nested->isNumeric())
|| (!!typeid_cast<const DataTypeString *>(type_default) != !!typeid_cast<const DataTypeString *>(type_arr_to_nested.get())))
throw Exception("Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (is_case_mode)
throw Exception{"CASE constructions must satisfy either of the following two conditions: "
"1. CASE clause and THEN clauses have common type T; "
"ELSE clause and WHEN clauses have common type U; "
"2. The CASE construction has no ELSE clause; "
"All the clauses have common type T", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
else
throw Exception{"Function " + getName()
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// Берём наименьший общий тип для элементов массива значений to и для default-а.
return getSmallestCommonNumericType(*type_arr_to_nested, *type_default);
DataTypeTraits::EnrichedDataTypePtr res = getSmallestCommonNumericType(enriched_type_arr_to_nested, *type_default);
return res.first;
}
/// TODO Больше проверок.
@ -130,7 +197,13 @@ public:
const ColumnConstArray * array_to = typeid_cast<const ColumnConstArray *>(&*block.getByPosition(arguments[2]).column);
if (!array_from || !array_to)
throw Exception("Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"WHEN clauses and THEN clauses in CASE construction "
"must provide constant data", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Second and third arguments of function " + getName() + " must be constant arrays.", ErrorCodes::ILLEGAL_COLUMN};
}
prepare(array_from->getData(), array_to->getData(), block, arguments);
@ -160,9 +233,15 @@ public:
&& !executeNum<Float32>(in, out, default_column)
&& !executeNum<Float64>(in, out, default_column)
&& !executeString(in, out, default_column))
throw Exception(
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal parameter in the CASE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
block.getByPosition(result).column = column_result;
}
@ -204,10 +283,16 @@ private:
{
auto out = typeid_cast<ColumnVector<T> *>(out_untyped);
if (!out)
throw Exception(
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal column " + out_untyped->getName() + "provided to THEN clauses"
" in CASE construction. Must be " + in->getName(), ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + out_untyped->getName() + " of elements of array of third argument of function " + getName()
+ ", must be " + in->getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplNumToNum<T>(in->getData(), out->getData());
}
@ -224,9 +309,16 @@ private:
&& !executeNumToNumWithConstDefault<T, Float32>(in, out_untyped)
&& !executeNumToNumWithConstDefault<T, Float64>(in, out_untyped)
&& !executeNumToStringWithConstDefault<T>(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else
{
@ -241,9 +333,16 @@ private:
&& !executeNumToNumWithNonConstDefault<T, Float32>(in, out_untyped, default_untyped)
&& !executeNumToNumWithNonConstDefault<T, Float64>(in, out_untyped, default_untyped)
&& !executeNumToStringWithNonConstDefault<T>(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
return true;
@ -259,9 +358,16 @@ private:
if (!default_untyped)
{
if (!executeStringToString(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else if (default_untyped->isConst())
{
@ -276,9 +382,16 @@ private:
&& !executeStringToNumWithConstDefault<Float32>(in, out_untyped)
&& !executeStringToNumWithConstDefault<Float64>(in, out_untyped)
&& !executeStringToStringWithConstDefault(in, out_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
else
{
@ -293,9 +406,16 @@ private:
&& !executeStringToNumWithNonConstDefault<Float32>(in, out_untyped, default_untyped)
&& !executeStringToNumWithNonConstDefault<Float64>(in, out_untyped, default_untyped)
&& !executeStringToStringWithNonConstDefault(in, out_untyped, default_untyped))
throw Exception(
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{
"Illegal column " + in->getName() + " provided to WHEN clauses in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{
"Illegal column " + in->getName() + " of elements of array of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
}
return true;
@ -332,9 +452,16 @@ private:
&& !executeNumToNumWithNonConstDefault2<T, U, Int64>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float32>(in, out, default_untyped)
&& !executeNumToNumWithNonConstDefault2<T, U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction",
ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
return true;
}
@ -372,8 +499,14 @@ private:
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplNumToStringWithNonConstDefault<T>(
in->getData(),
@ -411,9 +544,14 @@ private:
&& !executeStringToNumWithNonConstDefault2<U, Int64>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float32>(in, out, default_untyped)
&& !executeStringToNumWithNonConstDefault2<U, Float64>(in, out, default_untyped))
throw Exception(
"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
}
@ -459,8 +597,14 @@ private:
auto default_col = typeid_cast<const ColumnString *>(default_untyped);
if (!default_col)
throw Exception("Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
{
if (is_case_mode)
throw Exception{"Illegal column " + default_untyped->getName()
+ " provided to the ELSE clause in CASE construction", ErrorCodes::ILLEGAL_COLUMN};
else
throw Exception{"Illegal column " + default_untyped->getName() + " of fourth argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
executeImplStringToStringWithNonConstDefault(
in->getChars(), in->getOffsets(),
@ -713,7 +857,12 @@ private:
const size_t size = from.size();
if (0 == size)
throw Exception("Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS);
{
if (is_case_mode)
throw Exception{"CASE constructions require WHEN and THEN clauses", ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Empty arrays are illegal in function " + getName(), ErrorCodes::BAD_ARGUMENTS};
}
std::lock_guard<std::mutex> lock(mutex);
@ -721,7 +870,12 @@ private:
return;
if (from.size() != to.size())
throw Exception("Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS);
{
if (is_case_mode)
throw Exception{"Imbalance between WHEN and THEN clauses", ErrorCodes::BAD_ARGUMENTS};
else
throw Exception{"Second and third arguments of function " + getName() + " must be arrays of same size", ErrorCodes::BAD_ARGUMENTS};
}
Array converted_to;
const Array * used_to = &to;
@ -806,6 +960,9 @@ private:
prepared = true;
}
private:
bool is_case_mode = false;
};
}

View File

@ -12,7 +12,7 @@
#include <boost/mpl/comparison.hpp>
#include <DB/Core/Types.h>
#include <tuple>
namespace DB
{
@ -23,253 +23,480 @@ namespace DB
namespace NumberTraits
{
typedef boost::mpl::false_ Unsigned;
typedef boost::mpl::true_ Signed;
typedef boost::mpl::false_ Integer;
typedef boost::mpl::true_ Floating;
typedef boost::mpl::false_ Unsigned;
typedef boost::mpl::true_ Signed;
typedef boost::mpl::int_<0> Bits0;
typedef boost::mpl::int_<8> Bits8;
typedef boost::mpl::int_<16> Bits16;
typedef boost::mpl::int_<32> Bits32;
typedef boost::mpl::int_<64> Bits64;
typedef boost::mpl::int_<1024> BitsTooMany;
struct Error {};
typedef boost::mpl::false_ Integer;
typedef boost::mpl::true_ Floating;
template <typename T> struct Next;
typedef boost::mpl::int_<0> Bits0;
typedef boost::mpl::int_<8> Bits8;
typedef boost::mpl::int_<16> Bits16;
typedef boost::mpl::int_<32> Bits32;
typedef boost::mpl::int_<64> Bits64;
typedef boost::mpl::int_<1024> BitsTooMany;
template <> struct Next<Bits0> { typedef Bits0 Type; };
template <> struct Next<Bits8> { typedef Bits16 Type; };
template <> struct Next<Bits16> { typedef Bits32 Type; };
template <> struct Next<Bits32> { typedef Bits64 Type; };
template <> struct Next<Bits64> { typedef Bits64 Type; };
template <typename T> struct ExactNext { typedef typename Next<T>::Type Type; };
template <> struct ExactNext<Bits64> { typedef BitsTooMany Type; };
struct Error {};
template <typename T> struct Traits;
template <typename T> struct Next;
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; };
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; };
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; };
template <> struct Next<Bits0> { typedef Bits0 Type; };
template <> struct Next<Bits8> { typedef Bits16 Type; };
template <> struct Next<Bits16> { typedef Bits32 Type; };
template <> struct Next<Bits32> { typedef Bits64 Type; };
template <> struct Next<Bits64> { typedef Bits64 Type; };
template <typename Sign, typename Floatness, typename Bits> struct Construct;
template <typename T> struct ExactNext { typedef typename Next<T>::Type Type; };
template <> struct ExactNext<Bits64> { typedef BitsTooMany Type; };
template <> struct Construct<Unsigned, Integer, Bits0> { typedef void Type; };
template <> struct Construct<Unsigned, Floating, Bits0> { typedef void Type; };
template <> struct Construct<Signed, Integer, Bits0> { typedef void Type; };
template <> struct Construct<Signed, Floating, Bits0> { typedef void Type; };
template <> struct Construct<Unsigned, Integer, Bits8> { typedef UInt8 Type; };
template <> struct Construct<Unsigned, Integer, Bits16> { typedef UInt16 Type; };
template <> struct Construct<Unsigned, Integer, Bits32> { typedef UInt32 Type; };
template <> struct Construct<Unsigned, Integer, Bits64> { typedef UInt64 Type; };
template <> struct Construct<Unsigned, Floating, Bits8> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits16> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits32> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits64> { typedef Float64 Type; };
template <> struct Construct<Signed, Integer, Bits8> { typedef Int8 Type; };
template <> struct Construct<Signed, Integer, Bits16> { typedef Int16 Type; };
template <> struct Construct<Signed, Integer, Bits32> { typedef Int32 Type; };
template <> struct Construct<Signed, Integer, Bits64> { typedef Int64 Type; };
template <> struct Construct<Signed, Floating, Bits8> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits16> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits32> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits64> { typedef Float64 Type; };
template <typename Sign, typename Floatness> struct Construct<Sign, Floatness, BitsTooMany> { typedef Error Type; };
template <typename T>
inline bool isErrorType()
{
return false;
}
template <>
inline bool isErrorType<Error>()
{
return true;
}
template <typename T> struct Traits;
/** Результат сложения или умножения вычисляется по следующим правилам:
* - если один из аргументов с плавающей запятой, то результат - с плавающей запятой, иначе - целый;
* - если одно из аргументов со знаком, то результат - со знаком, иначе - без знака;
* - результат содержит больше бит (не только значащих), чем максимум в аргументах
* (например, UInt8 + Int32 = Int64).
*/
template <typename A, typename B> struct ResultOfAdditionMultiplication
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
};
template <> struct Traits<void> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits0 Bits; };
template <> struct Traits<UInt8> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<UInt16> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<UInt32> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<UInt64> { typedef Unsigned Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Int8> { typedef Signed Sign; typedef Integer Floatness; typedef Bits8 Bits; };
template <> struct Traits<Int16> { typedef Signed Sign; typedef Integer Floatness; typedef Bits16 Bits; };
template <> struct Traits<Int32> { typedef Signed Sign; typedef Integer Floatness; typedef Bits32 Bits; };
template <> struct Traits<Int64> { typedef Signed Sign; typedef Integer Floatness; typedef Bits64 Bits; };
template <> struct Traits<Float32> { typedef Signed Sign; typedef Floating Floatness; typedef Bits32 Bits; };
template <> struct Traits<Float64> { typedef Signed Sign; typedef Floating Floatness; typedef Bits64 Bits; };
template <typename A, typename B> struct ResultOfSubtraction
{
typedef typename Construct<
Signed,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
};
template <typename Sign, typename Floatness, typename Bits> struct Construct;
/** При делении всегда получается число с плавающей запятой.
*/
template <typename A, typename B> struct ResultOfFloatingPointDivision
{
typedef Float64 Type;
};
template <> struct Construct<Unsigned, Integer, Bits0> { typedef void Type; };
template <> struct Construct<Unsigned, Floating, Bits0> { typedef void Type; };
template <> struct Construct<Signed, Integer, Bits0> { typedef void Type; };
template <> struct Construct<Signed, Floating, Bits0> { typedef void Type; };
template <> struct Construct<Unsigned, Integer, Bits8> { typedef UInt8 Type; };
template <> struct Construct<Unsigned, Integer, Bits16> { typedef UInt16 Type; };
template <> struct Construct<Unsigned, Integer, Bits32> { typedef UInt32 Type; };
template <> struct Construct<Unsigned, Integer, Bits64> { typedef UInt64 Type; };
template <> struct Construct<Unsigned, Floating, Bits8> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits16> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits32> { typedef Float32 Type; };
template <> struct Construct<Unsigned, Floating, Bits64> { typedef Float64 Type; };
template <> struct Construct<Signed, Integer, Bits8> { typedef Int8 Type; };
template <> struct Construct<Signed, Integer, Bits16> { typedef Int16 Type; };
template <> struct Construct<Signed, Integer, Bits32> { typedef Int32 Type; };
template <> struct Construct<Signed, Integer, Bits64> { typedef Int64 Type; };
template <> struct Construct<Signed, Floating, Bits8> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits16> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits32> { typedef Float32 Type; };
template <> struct Construct<Signed, Floating, Bits64> { typedef Float64 Type; };
template <typename Sign, typename Floatness> struct Construct<Sign, Floatness, BitsTooMany> { typedef Error Type; };
/** При целочисленном делении получается число, битность которого равна делимому.
*/
template <typename A, typename B> struct ResultOfIntegerDivision
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Traits<A>::Bits>::Type Type;
};
template <typename T>
inline bool isErrorType()
{
return false;
}
template <>
inline bool isErrorType<Error>()
{
return true;
}
/** При взятии остатка получается число, битность которого равна делителю.
*/
template <typename A, typename B> struct ResultOfModulo
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename Traits<B>::Bits>::Type Type;
};
/** Результат сложения или умножения вычисляется по следующим правилам:
* - если один из аргументов с плавающей запятой, то результат - с плавающей запятой, иначе - целый;
* - если одно из аргументов со знаком, то результат - со знаком, иначе - без знака;
* - результат содержит больше бит (не только значащих), чем максимум в аргументах
* (например, UInt8 + Int32 = Int64).
*/
template <typename A, typename B> struct ResultOfAdditionMultiplication
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
};
template <typename A> struct ResultOfNegate
{
typedef typename Construct<
Signed,
typename Traits<A>::Floatness,
typename boost::mpl::if_<
typename Traits<A>::Sign,
typename Traits<A>::Bits,
typename Next<typename Traits<A>::Bits>::Type>::type>::Type Type;
};
template <typename A, typename B> struct ResultOfSubtraction
{
typedef typename Construct<
Signed,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Next<typename boost::mpl::max<typename Traits<A>::Bits, typename Traits<B>::Bits>::type>::Type>::Type Type;
};
template <typename A> struct ResultOfAbs
{
typedef typename Construct<
Unsigned,
typename Traits<A>::Floatness,
typename Traits <A>::Bits>::Type Type;
};
/** При делении всегда получается число с плавающей запятой.
*/
template <typename A, typename B> struct ResultOfFloatingPointDivision
{
typedef Float64 Type;
};
/** При побитовых операциях получается целое число, битность которого равна максимальной из битностей аргументов.
*/
template <typename A, typename B> struct ResultOfBit
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Floatness,
Bits64,
typename Traits<A>::Bits>::type,
typename boost::mpl::if_<
typename Traits<B>::Floatness,
Bits64,
typename Traits<B>::Bits>::type>::type>::Type Type;
};
/** При целочисленном делении получается число, битность которого равна делимому.
*/
template <typename A, typename B> struct ResultOfIntegerDivision
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
typename boost::mpl::or_<typename Traits<A>::Floatness, typename Traits<B>::Floatness>::type,
typename Traits<A>::Bits>::Type Type;
};
template <typename A> struct ResultOfBitNot
{
typedef typename Construct<
/** При взятии остатка получается число, битность которого равна делителю.
*/
template <typename A, typename B> struct ResultOfModulo
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename Traits<B>::Bits>::Type Type;
};
template <typename A> struct ResultOfNegate
{
typedef typename Construct<
Signed,
typename Traits<A>::Floatness,
typename boost::mpl::if_<
typename Traits<A>::Sign,
Integer,
typename Traits<A>::Bits>::Type Type;
};
/** Приведение типов для функции if:
* 1) void, Type -> Type
* 2) UInt<x>, UInt<y> -> UInt<max(x,y)>
* 3) Int<x>, Int<y> -> Int<max(x,y)>
* 4) Float<x>, Float<y> -> Float<max(x, y)>
* 5) UInt<x>, Int<y> -> Int<max(x*2, y)>
* 6) Float<x>, [U]Int<y> -> Float<max(x, y*2)>
* 7) UInt64 , Int<x> -> Error
* 8) Float<x>, [U]Int64 -> Error
*/
template <typename A, typename B>
struct ResultOfIf
{
typedef
/// 1)
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<A>::Bits, Bits0>::type,
B,
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<B>::Bits, Bits0>::type,
A,
/// 4) и 6)
typename boost::mpl::if_<
typename boost::mpl::or_<
typename Traits<A>::Floatness,
typename Traits<B>::Floatness>::type,
typename Construct<
Signed,
Floating,
typename boost::mpl::max< /// Этот максимум нужен только потому что if_ всегда вычисляет все аргументы.
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Floatness,
typename Traits<A>::Bits,
typename ExactNext<typename Traits<A>::Bits>::Type>::type,
typename boost::mpl::if_<
typename Traits<B>::Floatness,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
Bits32>::type>::Type,
/// 2) и 3)
typename boost::mpl::if_<
typename boost::mpl::equal_to<
typename Traits<A>::Sign,
typename Traits<B>::Sign>::type,
typename boost::mpl::if_<
typename boost::mpl::less<
typename Traits<A>::Bits,
typename Traits<B>::Bits>::type,
B,
A>::type,
/// 5)
typename Construct<
Signed,
Integer,
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Sign,
typename Traits<A>::Bits,
typename ExactNext<typename Traits<A>::Bits>::Type>::type,
typename boost::mpl::if_<
typename Traits<B>::Sign,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type>::Type>::type>::type>::type>::type Type;
};
/** Перед применением оператора % и побитовых операций, операнды приводятся к целым числам. */
template <typename A> struct ToInteger
{
typedef typename Construct<
typename Traits<A>::Sign,
Integer,
typename Traits<A>::Bits,
typename Next<typename Traits<A>::Bits>::Type>::type>::Type Type;
};
template <typename A> struct ResultOfAbs
{
typedef typename Construct<
Unsigned,
typename Traits<A>::Floatness,
typename Traits <A>::Bits>::Type Type;
};
/** При побитовых операциях получается целое число, битность которого равна максимальной из битностей аргументов.
*/
template <typename A, typename B> struct ResultOfBit
{
typedef typename Construct<
typename boost::mpl::or_<typename Traits<A>::Sign, typename Traits<B>::Sign>::type,
Integer,
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Floatness,
Bits64,
typename Traits<A>::Bits>::type>::Type Type;
};
typename Traits<A>::Bits>::type,
typename boost::mpl::if_<
typename Traits<B>::Floatness,
Bits64,
typename Traits<B>::Bits>::type>::type>::Type Type;
};
template <typename A> struct ResultOfBitNot
{
typedef typename Construct<
typename Traits<A>::Sign,
Integer,
typename Traits<A>::Bits>::Type Type;
};
/** Приведение типов для функции if:
* 1) void, Type -> Type
* 2) UInt<x>, UInt<y> -> UInt<max(x,y)>
* 3) Int<x>, Int<y> -> Int<max(x,y)>
* 4) Float<x>, Float<y> -> Float<max(x, y)>
* 5) UInt<x>, Int<y> -> Int<max(x*2, y)>
* 6) Float<x>, [U]Int<y> -> Float<max(x, y*2)>
* 7) UInt64 , Int<x> -> Error
* 8) Float<x>, [U]Int64 -> Error
*/
template <typename A, typename B>
struct ResultOfIf
{
typedef
/// 1)
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<A>::Bits, Bits0>::type,
B,
typename boost::mpl::if_<
typename boost::mpl::equal_to<typename Traits<B>::Bits, Bits0>::type,
A,
/// 4) и 6)
typename boost::mpl::if_<
typename boost::mpl::or_<
typename Traits<A>::Floatness,
typename Traits<B>::Floatness>::type,
typename Construct<
Signed,
Floating,
typename boost::mpl::max< /// Этот максимум нужен только потому что if_ всегда вычисляет все аргументы.
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Floatness,
typename Traits<A>::Bits,
typename ExactNext<typename Traits<A>::Bits>::Type>::type,
typename boost::mpl::if_<
typename Traits<B>::Floatness,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type,
Bits32>::type>::Type,
/// 2) и 3)
typename boost::mpl::if_<
typename boost::mpl::equal_to<
typename Traits<A>::Sign,
typename Traits<B>::Sign>::type,
typename boost::mpl::if_<
typename boost::mpl::less<
typename Traits<A>::Bits,
typename Traits<B>::Bits>::type,
B,
A>::type,
/// 5)
typename Construct<
Signed,
Integer,
typename boost::mpl::max<
typename boost::mpl::if_<
typename Traits<A>::Sign,
typename Traits<A>::Bits,
typename ExactNext<typename Traits<A>::Bits>::Type>::type,
typename boost::mpl::if_<
typename Traits<B>::Sign,
typename Traits<B>::Bits,
typename ExactNext<typename Traits<B>::Bits>::Type>::type>::type>::Type>::type>::type>::type>::type Type;
};
/** Перед применением оператора % и побитовых операций, операнды приводятся к целым числам. */
template <typename A> struct ToInteger
{
typedef typename Construct<
typename Traits<A>::Sign,
Integer,
typename boost::mpl::if_<
typename Traits<A>::Floatness,
Bits64,
typename Traits<A>::Bits>::type>::Type Type;
};
/// Notes on type composition.
///
/// 0. Problem statement.
///
/// Type composition with ResultOfIf is not associative. Example:
/// (Int8 x UInt32) x Float32 = Int64 x Float32 = Error;
/// Int8 x (UInt32 x Float32) = Int8 x Float64 = Float64.
/// In order to sort out this issue, we design a slightly improved version
/// of ResultOfIf.
///
/// I. A more rigorous approach to ResultOfIf.
///
/// First we organize the set of types:
/// T = {Void,Int8,Int16,Int32,Int64,UInt8,UInt16,UInt32,UInt64,Float32,Float64}
/// as a POSET P with the partial order being such that for any t1,t2 ∈ T,
/// t1 < t2 if and only if the domain of values of t1 is included in the domain
/// of values of T2.
///
/// For each type t ∈ T, we define C(t) as the set of chains of the poset P whose
/// unique minimal element is T.
///
/// Now for any two types t1,t2 ∈ T, we define the POSET C(t1,t2) as the intersection
/// C(t1) ∩ C(t2).
///
/// Denote K(t1,t2) as the unique antichain of C(t1,t2) whose each element minimally
/// represents both t1 and t2. It is important to keep in mind that t1 and t2 are
/// *not* comparable.
///
/// For the most part, K(t1,t2) coincides with the result of the application of
/// ResultOfIf to t1 and t2. Nevertheless, for some particular combinations of t1
/// and t2, the map K returns one of the following two antichains: {Int32,Float32},
/// {Int64,Float64}.
///
/// From these observations, we conclude that the type system T and the composition
/// law ResultOfIf are not powerful enough to represent all the combinations of
/// elements of T. That is the reason why ResultOfIf is not associative.
///
/// II. Extending ResultOfIf.
///
/// Let's embed T into a larger set E of "enriched types" such that:
/// 1. E ⊂ TxT;
/// 2. for each t ∈ T, (T,Void) ∈ E.
/// 3. (Int32,Float32) ∈ E
/// 4. (Int64,Float64) ∈ E.
///
/// E represents the image A of the map K, a set of antichains, as a set of types.
///
/// Consider the canonical injection ψ : T x T ----> E x E and the natural bijection
/// φ : A ----> E.
/// Then there exists a unique map K' : E x E ----> E, that makes the diagram below
/// commutative:
///
/// K
/// T x T ----> A
/// | |
/// | ψ | φ
/// ↓ K' ↓
/// E x E ----> E
///
/// K' is exactly the same map as K, the sole difference being that K' takes as
/// parameters extended types that map to ordinary ones.
///
/// Finally we extend the map K' by taking into account the new types (Int32,Float32)
/// and (Int32,Float32). This extended map is called TypeProduct in the implementation.
/// TypeProduct is both commutative and associative.
///
/// III. Usage.
///
/// When we need to compose ordinary types, the following is to be performed:
/// 1. embed each type into its counterpart in E with EmbedType;
/// 2. compose the resulting enriched types with TypeProduct;
/// 3. return the first component of the result, which means that, given an extended
/// type e = (p,q) ∈ E, we return the ordinary type p ∈ T.
///
/// The result is the type we are looking for.
///
/// IV. Example.
///
/// Suppose we need to compose, as in the problem statement, the types Int8,
/// UInt32, and Float32. The corresponding embedded types are:
/// (Int8,Void), (UInt32,Void), (Float32,Void).
///
/// By computing (Int8 x UInt32) x Float32, we get:
///
/// TypeProduct(TypeProduct((Int8,Void),(UInt32,Void)), (Float32,Void))
/// = TypeProduct((Int64,Float64), (Float32,Void))
/// = (Float64,void)
/// Thus, (Int8 x UInt32) x Float32 = Float64.
///
/// By computing Int8 x (UInt32 x Float32), we get:
///
/// TypeProduct((Int8,Void), TypeProduct((UInt32,Void), (Float32,Void)))
/// = TypeProduct((Int8,Void), (Float64,Void))
/// = (Float64,void)
/// Thus, Int8 x (UInt32 x Float32) = Float64.
///
namespace Enriched
{
/// Definitions of enriched types.
using Void = std::tuple<void, void>;
using Int8 = std::tuple<DB::Int8, void>;
using Int16 = std::tuple<DB::Int16, void>;
using Int32 = std::tuple<DB::Int32, void>;
using Int64 = std::tuple<DB::Int64, void>;
using UInt8 = std::tuple<DB::UInt8, void>;
using UInt16 = std::tuple<DB::UInt16, void>;
using UInt32 = std::tuple<DB::UInt32, void>;
using UInt64 = std::tuple<DB::UInt64, void>;
using Float32 = std::tuple<DB::Float32, void>;
using Float64 = std::tuple<DB::Float64, void>;
using IntFloat32 = std::tuple<DB::Int32, DB::Float32>;
using IntFloat64 = std::tuple<DB::Int64, DB::Float64>;
}
/// Embed an ordinary type into the corresponding enriched type.
template <typename T>
struct EmbedType;
template <> struct EmbedType<void> { using Type = Enriched::Void; };
template <> struct EmbedType<Int8> { using Type = Enriched::Int8; };
template <> struct EmbedType<Int16> { using Type = Enriched::Int16; };
template <> struct EmbedType<Int32> { using Type = Enriched::Int32; };
template <> struct EmbedType<Int64> { using Type = Enriched::Int64; };
template <> struct EmbedType<UInt8> { using Type = Enriched::UInt8; };
template <> struct EmbedType<UInt16> { using Type = Enriched::UInt16; };
template <> struct EmbedType<UInt32> { using Type = Enriched::UInt32; };
template <> struct EmbedType<UInt64> { using Type = Enriched::UInt64; };
template <> struct EmbedType<Float32> { using Type = Enriched::Float32; };
template <> struct EmbedType<Float64> { using Type = Enriched::Float64; };
/// Get an ordinary type from an enriched type.
template <typename TType>
struct ToOrdinaryType
{
using Type = typename std::tuple_element<0, TType>::type;
};
/// Get an ordinary type from an enriched type.
/// Error case.
template <>
struct ToOrdinaryType<Error>
{
using Type = Error;
};
/// Compute the product of two enriched numeric types.
template <typename T1, typename T2>
struct TypeProduct
{
using Type = Error;
};
/// Compute the product of two enriched numeric types.
/// Case when both of the source types and the resulting type map to ordinary types.
template <typename A, typename B>
struct TypeProduct<std::tuple<A, void>, std::tuple<B, void> >
{
private:
using Result = typename NumberTraits::ResultOfIf<A, B>::Type;
public:
using Type = typename std::conditional<
std::is_same<Result, Error>::value,
Error,
std::tuple<Result, void>
>::type;
};
/// Compute the product of two enriched numeric types.
/// Case when a source type or the resulting type does not map to any ordinary type.
template <> struct TypeProduct<Enriched::Int8, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::Int8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int8, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int16, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::Int16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int16, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int32, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::Int32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int8, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::Int16, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int32> { using Type = Enriched::Int32; };
template <> struct TypeProduct<Enriched::Int32, Enriched::IntFloat32> { using Type = Enriched::Int32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Int64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::Int64, Enriched::IntFloat32> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Float32> { using Type = Enriched::Float32; };
template <> struct TypeProduct<Enriched::Float32, Enriched::IntFloat32> { using Type = Enriched::Float32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::Float64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float64, Enriched::IntFloat32> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt8> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt8, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt16> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::IntFloat32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::IntFloat32> { using Type = Enriched::IntFloat32; };
template <> struct TypeProduct<Enriched::IntFloat32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::IntFloat32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int8, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int16, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::Int32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Int64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::Int64, Enriched::IntFloat64> { using Type = Enriched::Int64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Float32> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float32, Enriched::IntFloat64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::Float64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::Float64, Enriched::IntFloat64> { using Type = Enriched::Float64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt8> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt8, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt16> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt16, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::UInt32> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::UInt32, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
template <> struct TypeProduct<Enriched::IntFloat64, Enriched::IntFloat64> { using Type = Enriched::IntFloat64; };
}
}

View File

@ -34,57 +34,32 @@ public:
FunctionKind kind{UNKNOWN};
enum class Genus
{
ORDINARY = 0,
CASE_WITH_EXPR,
CASE_WITHOUT_EXPR,
CASE_ARRAY
};
Genus genus{Genus::ORDINARY};
public:
ASTFunction() = default;
ASTFunction(const StringRange range_) : ASTWithAlias(range_) {}
String getColumnName() const override
{
String res;
WriteBufferFromString wb(res);
writeString(name, wb);
if (parameters)
{
writeChar('(', wb);
for (ASTs::const_iterator it = parameters->children.begin(); it != parameters->children.end(); ++it)
{
if (it != parameters->children.begin())
writeCString(", ", wb);
writeString((*it)->getColumnName(), wb);
}
writeChar(')', wb);
}
writeChar('(', wb);
for (ASTs::const_iterator it = arguments->children.begin(); it != arguments->children.end(); ++it)
{
if (it != arguments->children.begin())
writeCString(", ", wb);
writeString((*it)->getColumnName(), wb);
}
writeChar(')', wb);
return res;
}
String getColumnName() const override;
/** Получить текст, который идентифицирует этот элемент. */
String getID() const override { return "Function_" + name; }
String getID() const override;
ASTPtr clone() const override
{
ASTFunction * res = new ASTFunction(*this);
ASTPtr ptr{res};
res->children.clear();
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
return ptr;
}
ASTPtr clone() const override;
protected:
void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
private:
void formatCase(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const;
};

View File

@ -0,0 +1,21 @@
#pragma once
#include <DB/Parsers/ParserQueryWithOutput.h>
namespace DB
{
/** CASE construction
* Two variants:
* 1. CASE expr WHEN val1 THEN res1 [WHEN ...] ELSE resN END
* 2. CASE WHEN cond1 THEN res1 [WHEN ...] ELSE resN END
* NOTE Until we get full support for NULL values in ClickHouse, ELSE sections are mandatory.
*/
class ParserCase final : public IParserBase
{
protected:
const char * getName() const override { return "case"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) override;
};
}

View File

@ -346,6 +346,7 @@ namespace ErrorCodes
extern const int NO_ZOOKEEPER_ACCESSOR = 340;
extern const int UNFINISHED = 341;
extern const int METADATA_MISMATCH = 342;
extern const int INVALID_FUNCTION_GENUS = 343;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -18,11 +18,17 @@ namespace ErrorCodes
}
DataTypeArray::DataTypeArray(DataTypePtr nested_) : nested(nested_)
DataTypeArray::DataTypeArray(DataTypePtr nested_)
: enriched_nested(std::make_pair(nested_, new DataTypeVoid)), nested{nested_}
{
offsets = new DataTypeFromFieldType<ColumnArray::Offset_t>::Type;
}
DataTypeArray::DataTypeArray(DataTypeTraits::EnrichedDataTypePtr enriched_nested_)
: enriched_nested{enriched_nested_}, nested{enriched_nested.first}
{
offsets = new DataTypeFromFieldType<ColumnArray::Offset_t>::Type;
}
void DataTypeArray::serializeBinary(const Field & field, WriteBuffer & ostr) const
{

View File

@ -1,23 +1,16 @@
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/DataTypeTraits.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypeFixedString.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/DataTypeFromFieldTypeOrError.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace Conditional
{
@ -50,267 +43,98 @@ std::string dumpArgTypes(const DataTypes & args)
return out;
}
template <typename TType>
struct ExtendedDataType
{
using FieldType = typename TType::FieldType;
};
template <>
struct ExtendedDataType<void>
{
using FieldType = void;
};
/// Forward declarations.
template <typename TFloat, typename TUInt, typename TInt, typename TType>
template <typename TResult, typename TType>
struct ResultDataTypeDeducer;
template <typename TFloat, typename TUInt, typename TInt>
struct ResultDataTypeDeducerImpl;
template <typename TFloat, typename TUInt, typename TInt>
struct TypeComposer;
template <typename TType, typename TInt>
struct TypeComposerImpl;
/// Analyze the type of each branch (then, else) of a multiIf function.
/// Determine the returned type if all branches are numeric.
class FirstResultDataTypeDeducer final
/// Internal class used by ResultDataTypeDeducer. Calls ResultDataTypeDeducer
/// for the next branch to be processed.
template <typename TType>
class ResultDataTypeDeducerImpl final
{
private:
template <typename U>
using ConcreteDataTypeDeducer = ResultDataTypeDeducer<void, void, void, U>;
public:
static void execute(const DataTypes & args, DataTypePtr & type_res)
static void execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
size_t i = firstThen();
if (! (ResultDataTypeDeducer<TType, DataTypeUInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat64>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
if (!DataTypeDispatcher<ConcreteDataTypeDeducer>::apply(args, i, type_res))
throw Exception{"Illegal type of column " + toString(i) +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
/// Specialization for the error type.
template <>
class ResultDataTypeDeducerImpl<NumberTraits::Error> final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw CondException{CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR, dump};
}
};
/// Analyze the type of the branch currently being processed of a multiIf function.
/// Subsequently perform the same analysis for the remaining branches.
/// Determine the returned type if all the processed branches are numeric.
template <typename TFloat, typename TUInt, typename TInt, typename TType>
template <typename TResult, typename TType>
class ResultDataTypeDeducer final
{
private:
using TCombinedFloat = typename std::conditional<
std::is_floating_point<typename ExtendedDataType<TType>::FieldType>::value,
typename NumberTraits::ResultOfIf<
typename ExtendedDataType<TFloat>::FieldType,
typename ExtendedDataType<TType>::FieldType
>::Type,
typename ExtendedDataType<TFloat>::FieldType
>::type;
using TCombinedUInt = typename std::conditional<
std::is_unsigned<typename ExtendedDataType<TType>::FieldType>::value,
typename NumberTraits::ResultOfIf<
typename ExtendedDataType<TUInt>::FieldType,
typename ExtendedDataType<TType>::FieldType
>::Type,
typename ExtendedDataType<TUInt>::FieldType
>::type;
using TCombinedInt = typename std::conditional<
std::is_signed<typename ExtendedDataType<TType>::FieldType>::value,
typename NumberTraits::ResultOfIf<
typename ExtendedDataType<TInt>::FieldType,
typename ExtendedDataType<TType>::FieldType
>::Type,
typename ExtendedDataType<TInt>::FieldType
>::type;
using ConcreteComposer = TypeComposer<TCombinedFloat, TCombinedUInt, TCombinedInt>;
using DataTypeDeducerImpl = ResultDataTypeDeducerImpl<TCombinedFloat, TCombinedUInt, TCombinedInt>;
using TCombined = typename DataTypeTraits::DataTypeProduct<TResult, TType>::Type;
using DataTypeDeducerImpl = ResultDataTypeDeducerImpl<TCombined>;
public:
static bool execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
static bool execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
if (typeid_cast<const TType *>(&*args[i]) == nullptr)
return false;
if (i == elseArg(args))
{
ConcreteComposer::execute(args, i, type_res);
return true;
type_res = DataTypeTraits::ToEnrichedDataTypeObject<TCombined, false>::execute();
if ((type_res.first == DataTypePtr()) && (type_res.second == DataTypePtr()))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
else
{
i = std::min(nextThen(i), elseArg(args));
DataTypeDeducerImpl::execute(args, i, type_res);
}
i = std::min(nextThen(i), elseArg(args));
if (DataTypeDeducerImpl::execute(args, i, type_res))
return true;
else
throw Exception{"Illegal type of column " + toString(i) +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return true;
}
};
/// Internal class used by ResultDataTypeDeducer. From the deduced triplet
/// of numeric types (float, unsigned int, signed int), determine the
/// associated triplet of extended data types, then call ResultDataTypeDeducer
/// for the next branch to being processed.
template <typename TFloat, typename TUInt, typename TInt>
class ResultDataTypeDeducerImpl final
{
private:
using ExtendedDataTypeFloat = typename std::conditional<
std::is_void<TFloat>::value,
void,
typename DataTypeFromFieldType<TFloat>::Type
>::type;
using ExtendedDataTypeUInt = typename std::conditional<
std::is_void<TUInt>::value,
void,
typename DataTypeFromFieldType<TUInt>::Type
>::type;
using ExtendedDataTypeInt = typename std::conditional<
std::is_void<TInt>::value,
void,
typename DataTypeFromFieldType<TInt>::Type
>::type;
template <typename U>
using ConcreteDataTypeDeducer = ResultDataTypeDeducer<
ExtendedDataTypeFloat,
ExtendedDataTypeUInt,
ExtendedDataTypeInt,
U
>;
public:
static bool execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
if (DataTypeDispatcher<ConcreteDataTypeDeducer>::apply(args, i, type_res))
return true;
else
throw Exception{"Illegal type of column " + toString(i) +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for type deduction error.
template <typename TFloat, typename TUInt>
class ResultDataTypeDeducerImpl<TFloat, TUInt, typename NumberTraits::Error> final
/// Analyze the type of each branch (then, else) of a multiIf function.
/// Determine the returned type if all branches are numeric.
class FirstResultDataTypeDeducer final
{
public:
static bool execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
static void execute(const DataTypes & args, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
using Void = typename DataTypeTraits::ToEnrichedDataType<NumberTraits::Enriched::Void>::Type;
/// Specialization for type deduction error.
template <typename TFloat, typename TInt>
class ResultDataTypeDeducerImpl<TFloat, typename NumberTraits::Error, TInt> final
{
public:
static bool execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
size_t i = firstThen();
/// Specialization for type deduction error.
template <typename TUInt, typename TInt>
class ResultDataTypeDeducerImpl<typename NumberTraits::Error, TUInt, TInt> final
{
public:
static bool execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Compose a float type, an unsigned int type, and a signed int type.
/// Return the deduced type.
template <typename TFloat, typename TUInt, typename TInt>
class TypeComposer final
{
private:
using TCombined = typename NumberTraits::ResultOfIf<TFloat, TUInt>::Type;
public:
static void execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
TypeComposerImpl<TCombined, TInt>::execute(args, i, type_res);
}
};
template <typename TType, typename TInt>
class TypeComposerImpl final
{
private:
using TCombined = typename NumberTraits::ResultOfIf<TType, TInt>::Type;
public:
static void execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
DataTypePtr res = DataTypeFromFieldTypeOrError<TCombined>::getDataType();
if (!res)
throw Exception{"Illegal type of column " + toString(i) +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
type_res = res;
}
};
/// Specialization for type composition error.
template <typename TInt>
class TypeComposerImpl<typename NumberTraits::Error, TInt> final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for type composition error.
template <typename TType>
class TypeComposerImpl<TType, typename NumberTraits::Error> final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for type composition error.
template <>
class TypeComposerImpl<typename NumberTraits::Error, typename NumberTraits::Error> final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw Exception{"Arguments of function multiIf are not upscalable to a "
"common type without loss of precision: " + dump,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (! (ResultDataTypeDeducer<Void, DataTypeUInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat64>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
@ -318,9 +142,9 @@ public:
DataTypePtr getReturnTypeForArithmeticArgs(const DataTypes & args)
{
DataTypePtr type_res;
DataTypeTraits::EnrichedDataTypePtr type_res;
FirstResultDataTypeDeducer::execute(args, type_res);
return type_res;
return type_res.first;
}
bool hasArithmeticBranches(const DataTypes & args)
@ -404,8 +228,7 @@ bool hasFixedStringsOfIdenticalLength(const DataTypes & args)
{
auto fixed_str = typeid_cast<const DataTypeFixedString *>(args[i].get());
if (fixed_str == nullptr)
throw Exception{"Illegal type of column " + toString(i) +
" of function multiIf", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
return fixed_str->getN();
};

View File

@ -1,4 +1,5 @@
#include <DB/Functions/Conditional/CondSource.h>
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnConst.h>
@ -50,9 +51,8 @@ const PaddedPODArray<UInt8> & CondSource::initDataArray(const Block & block, con
const auto * vec_col = typeid_cast<const ColumnVector<UInt8> *>(source_col);
if (vec_col == nullptr)
throw Exception{"Illegal column " + source_col->getName() + " of argument "
+ toString(i) + " of function multiIf."
"Must be ColumnUInt8 or ColumnConstUInt8.", ErrorCodes::ILLEGAL_COLUMN};
throw CondException{CondErrorCodes::COND_SOURCE_ILLEGAL_COLUMN,
source_col->getName(), toString(i)};
return vec_col->getData();
}

View File

@ -1,9 +1,10 @@
#include <DB/Functions/Conditional/NumericPerformer.h>
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/NumericEvaluator.h>
#include <DB/Functions/Conditional/ArrayEvaluator.h>
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/DataTypeFromFieldTypeOrError.h>
#include <DB/Functions/DataTypeTraits.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/Columns/ColumnVector.h>
@ -28,99 +29,6 @@ namespace Conditional
namespace
{
/// Forward declarations.
template <typename TFloat, typename TUInt, typename TInt>
struct ElseFunctor;
template <typename TType, typename TInt>
struct ElseFunctorImpl;
/// Perform the multiIf function for either numeric or array branch parameters,
/// after we successfully processed the Else branch.
template <typename TFloat, typename TUInt, typename TInt>
struct ElseFunctor final
{
private:
using TCombined = typename NumberTraits::ResultOfIf<TFloat, TUInt>::Type;
public:
static void executeNumeric(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
ElseFunctorImpl<TCombined, TInt>::executeNumeric(branches, block, args, result);
}
static void executeArray(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
ElseFunctorImpl<TCombined, TInt>::executeArray(branches, block, args, result);
}
};
template <typename TType, typename TInt>
struct ElseFunctorImpl final
{
private:
using TCombined = typename NumberTraits::ResultOfIf<TType, TInt>::Type;
public:
static void executeNumeric(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
NumericEvaluator<TCombined>::perform(branches, block, args, result);
}
static void executeArray(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
ArrayEvaluator<TCombined>::perform(branches, block, args, result);
}
};
/// Specialization for type composition error.
template <typename TInt>
class ElseFunctorImpl<typename NumberTraits::Error, TInt> final
{
public:
static void executeNumeric(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
static void executeArray(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for type composition error.
template <typename TType>
class ElseFunctorImpl<TType, typename NumberTraits::Error> final
{
public:
static void executeNumeric(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
static void executeArray(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for type composition error.
template <>
class ElseFunctorImpl<typename NumberTraits::Error, typename NumberTraits::Error> final
{
public:
static void executeNumeric(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
static void executeArray(const Branches & branches, Block & block, const ColumnNumbers & args, size_t result)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// This class provides a means to collect type information on a branch
/// (then or else) of a multiIf function.
template <typename TType>
@ -194,7 +102,7 @@ protected:
}
branch.index = index;
branch.type = DataTypeFromFieldTypeOrError<TType>::getDataType();
branch.type = DataTypeTraits::DataTypeFromFieldTypeOrError<TType>::getDataType();
branches.push_back(branch);
@ -209,28 +117,17 @@ protected:
/// Collect type information on the unique else branch of a multiIf function.
/// Determine the returned value type. Perform the multiIf.
template <typename TFloat, typename TUInt, typename TInt, typename TType>
template <typename TResult, typename TType>
struct ElsePredicate final : public PredicateBase<TType>
{
using Base = PredicateBase<TType>;
using TCombinedFloat = typename std::conditional<
std::is_floating_point<TType>::value,
typename NumberTraits::ResultOfIf<TFloat, TType>::Type,
TFloat
>::type;
using TCombined = typename NumberTraits::TypeProduct<
TResult,
typename NumberTraits::EmbedType<TType>::Type
>::Type;
using TCombinedUInt = typename std::conditional<
std::is_unsigned<TType>::value,
typename NumberTraits::ResultOfIf<TUInt, TType>::Type,
TUInt
>::type;
using TCombinedInt = typename std::conditional<
std::is_signed<TType>::value,
typename NumberTraits::ResultOfIf<TInt, TType>::Type,
TInt
>::type;
using TFinal = typename NumberTraits::ToOrdinaryType<TCombined>::Type;
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -242,9 +139,9 @@ struct ElsePredicate final : public PredicateBase<TType>
if (category == Base::NONE)
return false;
else if (category == Base::NUMERIC)
ElseFunctor<TCombinedFloat, TCombinedUInt, TCombinedInt>::executeNumeric(branches, block, args, result);
NumericEvaluator<TFinal>::perform(branches, block, args, result);
else if (category == Base::NUMERIC_ARRAY)
ElseFunctor<TCombinedFloat, TCombinedUInt, TCombinedInt>::executeArray(branches, block, args, result);
ArrayEvaluator<TFinal>::perform(branches, block, args, result);
else
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
@ -253,20 +150,8 @@ struct ElsePredicate final : public PredicateBase<TType>
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TUInt, typename TInt>
struct ElsePredicate<TFloat, TUInt, TInt, NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TUInt, typename TType>
struct ElsePredicate<TFloat, TUInt, NumberTraits::Error, TType>
template <typename TResult>
struct ElsePredicate<TResult, NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -276,19 +161,8 @@ struct ElsePredicate<TFloat, TUInt, NumberTraits::Error, TType>
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TInt, typename TType>
struct ElsePredicate<TFloat, NumberTraits::Error, TInt, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for incorrect type information.
template <typename TUInt, typename TInt, typename TType>
struct ElsePredicate<NumberTraits::Error, TUInt, TInt, TType>
template <typename TType>
struct ElsePredicate<NumberTraits::Error, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -299,34 +173,15 @@ struct ElsePredicate<NumberTraits::Error, TUInt, TInt, TType>
/// Collect type information on a then branch of a multiIf function.
/// Update the returned value type information. Go to the next branch.
template <typename TFloat, typename TUInt, typename TInt, typename TType>
template <typename TResult, typename TType>
struct ThenPredicate final : public PredicateBase<TType>
{
using Base = PredicateBase<TType>;
using TCombinedFloat = typename std::conditional<
std::is_floating_point<TType>::value,
typename NumberTraits::ResultOfIf<TFloat, TType>::Type,
TFloat
>::type;
using TCombinedUInt = typename std::conditional<
std::is_unsigned<TType>::value,
typename NumberTraits::ResultOfIf<TUInt, TType>::Type,
TUInt
>::type;
using TCombinedInt = typename std::conditional<
std::is_signed<TType>::value,
typename NumberTraits::ResultOfIf<TInt, TType>::Type,
TInt
>::type;
template <typename U>
using ConcreteThenPredicate = ThenPredicate<TCombinedFloat, TCombinedUInt, TCombinedInt, U>;
template <typename U>
using ConcreteElsePredicate = ElsePredicate<TCombinedFloat, TCombinedUInt, TCombinedInt, U>;
using TCombined = typename NumberTraits::TypeProduct<
TResult,
typename NumberTraits::EmbedType<TType>::Type
>::Type;
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -342,18 +197,34 @@ struct ThenPredicate final : public PredicateBase<TType>
if (index2 != elseArg(args))
{
/// We have a pair Cond-Then. Process the next Then.
if (!NumericTypeDispatcher<ConcreteThenPredicate>::apply(index2 + 1,
block, args, result, branches))
throw Exception{"Illegal column " + toString(index2 + 1) +
" of function multiIf", ErrorCodes::ILLEGAL_COLUMN};
if (! (ThenPredicate<TCombined, UInt8>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt16>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, UInt64>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int8>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int16>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Int64>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Float32>::execute(index2 + 1, block, args, result, branches)
|| ThenPredicate<TCombined, Float64>::execute(index2 + 1, block, args, result, branches)))
throw CondException{CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN,
toString(index2 + 1)};
}
else
{
/// We have an Else which ends the multiIf. Process it.
if (!NumericTypeDispatcher<ConcreteElsePredicate>::apply(index2,
block, args, result, branches))
throw Exception{"Illegal column " + toString(index2) +
" of function multiIf", ErrorCodes::ILLEGAL_COLUMN};
if (! (ElsePredicate<TCombined, UInt8>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt16>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, UInt64>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int8>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int16>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Int64>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Float32>::execute(index2, block, args, result, branches)
|| ElsePredicate<TCombined, Float64>::execute(index2, block, args, result, branches)))
throw CondException{CondErrorCodes::NUMERIC_PERFORMER_ILLEGAL_COLUMN,
toString(index2)};
}
return true;
@ -361,20 +232,8 @@ struct ThenPredicate final : public PredicateBase<TType>
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TUInt, typename TInt>
struct ThenPredicate<TFloat, TUInt, TInt, NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TUInt, typename TType>
struct ThenPredicate<TFloat, TUInt, NumberTraits::Error, TType>
template <typename TResult>
struct ThenPredicate<TResult, NumberTraits::Error>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -384,19 +243,8 @@ struct ThenPredicate<TFloat, TUInt, NumberTraits::Error, TType>
};
/// Specialization for incorrect type information.
template <typename TFloat, typename TInt, typename TType>
struct ThenPredicate<TFloat, NumberTraits::Error, TInt, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
{
throw Exception{"Internal logic error", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
};
/// Specialization for incorrect type information.
template <typename TUInt, typename TInt, typename TType>
struct ThenPredicate<NumberTraits::Error, TUInt, TInt, TType>
template <typename TType>
struct ThenPredicate<NumberTraits::Error, TType>
{
static bool execute(size_t index, Block & block, const ColumnNumbers & args,
size_t result, Branches & branches)
@ -408,13 +256,21 @@ struct ThenPredicate<NumberTraits::Error, TUInt, TInt, TType>
/// First Then
struct FirstThenPredicate final
{
template <typename U>
using ConcreteThenPredicate = ThenPredicate<void, void, void, U>;
static bool execute(Block & block, const ColumnNumbers & args, size_t result)
{
using Void = NumberTraits::Enriched::Void;
Branches branches;
return NumericTypeDispatcher<ConcreteThenPredicate>::apply(firstThen(), block, args, result, branches);
return ThenPredicate<Void, UInt8>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt16>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, UInt64>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int8>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int16>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Int64>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Float32>::execute(firstThen(), block, args, result, branches)
|| ThenPredicate<Void, Float64>::execute(firstThen(), block, args, result, branches);
}
};

View File

@ -7,6 +7,7 @@
#include <DB/Columns/ColumnVector.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnFixedString.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/Core/Types.h>
namespace DB

View File

@ -0,0 +1,144 @@
#include <DB/Functions/Conditional/ArgsInfo.h>
#include <DB/Functions/Conditional/CondException.h>
#include <DB/Functions/Conditional/common.h>
#include <DB/Functions/DataTypeTraits.h>
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/WriteHelpers.h>
namespace DB
{
namespace Conditional
{
namespace
{
std::string dumpArgTypes(const DataTypes & args)
{
std::string out;
WriteBufferFromString buf{out};
bool is_first = true;
for (size_t i = 0; i < args.size(); ++i)
{
if (is_first)
is_first = false;
else
writeString("; ", buf);
writeString(args[i]->getName(), buf);
}
buf.next();
return out;
}
/// Forward declarations.
template <typename TResult, typename TType>
struct ResultDataTypeDeducer;
/// Internal class used by ResultDataTypeDeducer. Calls ResultDataTypeDeducer
/// for the next element to be processed.
template <typename TType>
class ResultDataTypeDeducerImpl final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
if (! (ResultDataTypeDeducer<TType, DataTypeUInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeUInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, DataTypeFloat64>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
/// Specialization for the error type.
template <>
class ResultDataTypeDeducerImpl<typename NumberTraits::Error> final
{
public:
static void execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
std::string dump = dumpArgTypes(args);
throw CondException{CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR, dump};
}
};
/// Analyze the type of the element currently being processed of an array.
/// Subsequently perform the same analysis for the remaining elements.
/// Determine the returned type if all the processed elements are numeric.
template <typename TResult, typename TType>
class ResultDataTypeDeducer final
{
private:
using TCombined = typename DataTypeTraits::DataTypeProduct<TResult, TType>::Type;
using DataTypeDeducerImpl = ResultDataTypeDeducerImpl<TCombined>;
public:
static bool execute(const DataTypes & args, size_t i, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
if (typeid_cast<const TType *>(&*args[i]) == nullptr)
return false;
if (i == (args.size() - 1))
{
type_res = DataTypeTraits::ToEnrichedDataTypeObject<TCombined, false>::execute();
if ((type_res.first == DataTypePtr()) && (type_res.second == DataTypePtr()))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
else
{
++i;
DataTypeDeducerImpl::execute(args, i, type_res);
}
return true;
}
};
/// Analyze the type of each element of an array.
/// Determine the returned type if all elements are numeric.
class FirstResultDataTypeDeducer final
{
public:
static void execute(const DataTypes & args, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
using Void = typename DataTypeTraits::ToEnrichedDataType<NumberTraits::Enriched::Void>::Type;
size_t i = 0;
if (! (ResultDataTypeDeducer<Void, DataTypeUInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeUInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt8>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt16>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeInt64>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat32>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, DataTypeFloat64>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
}
DataTypeTraits::EnrichedDataTypePtr getArrayType(const DataTypes & args)
{
DataTypeTraits::EnrichedDataTypePtr type_res;
FirstResultDataTypeDeducer::execute(args, type_res);
return type_res;
}
}
}

View File

@ -1,71 +1,89 @@
#include <DB/Functions/NumberTraits.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsTransform.h>
#include <DB/Functions/DataTypeTraits.h>
namespace DB
{
/// TODO: Убрать copy-paste из FunctionsConditional.h
template <typename T>
struct DataTypeFromFieldTypeOrError
namespace
{
static DataTypePtr getDataType()
template <typename TLeft, typename TRight, typename TType>
struct TypeProcessorImpl
{
static DataTypeTraits::EnrichedDataTypePtr execute()
{
return new typename DataTypeFromFieldType<T>::Type;
using EnrichedT1 = std::tuple<TLeft, TRight>;
using EnrichedT2 = typename NumberTraits::EmbedType<TType>::Type;
using TCombined = typename NumberTraits::TypeProduct<EnrichedT1, EnrichedT2>::Type;
auto type_res = DataTypeTraits::ToEnrichedDataTypeObject<TCombined, true>::execute();
if ((type_res.first == DataTypePtr()) && (type_res.second == DataTypePtr()))
throw Exception("Types " + TypeName<TLeft>::get() + " and " + TypeName<TType>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
};
template <>
struct DataTypeFromFieldTypeOrError<NumberTraits::Error>
template <typename TLeft, typename TRight>
struct RightTypeProcessor
{
static DataTypePtr getDataType()
static DataTypeTraits::EnrichedDataTypePtr execute(const IDataType & type2)
{
return nullptr;
if (typeid_cast<const DataTypeUInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt8>::execute();
if (typeid_cast<const DataTypeUInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt16>::execute();
if (typeid_cast<const DataTypeUInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt32>::execute();
if (typeid_cast<const DataTypeUInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt64>::execute();
if (typeid_cast<const DataTypeInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int8>::execute();
if (typeid_cast<const DataTypeInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int16>::execute();
if (typeid_cast<const DataTypeInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int32>::execute();
if (typeid_cast<const DataTypeInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int64>::execute();
if (typeid_cast<const DataTypeFloat32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float32>::execute();
if (typeid_cast<const DataTypeFloat64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float64>::execute();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
};
template <typename T1, typename T2>
DataTypePtr getSmallestCommonNumericTypeImpl()
template <typename TLeft>
struct LeftTypeProcessor
{
using ResultType = typename NumberTraits::ResultOfIf<T1, T2>::Type;
auto type_res = DataTypeFromFieldTypeOrError<ResultType>::getDataType();
if (!type_res)
throw Exception("Types " + TypeName<T1>::get() + " and " + TypeName<T2>::get()
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
static DataTypeTraits::EnrichedDataTypePtr execute(const DataTypePtr & right, const IDataType & type2)
{
if (typeid_cast<const DataTypeVoid *>(&*right)) return RightTypeProcessor<TLeft, void>::execute(type2);
if (typeid_cast<const DataTypeUInt8 *>(&*right)) return RightTypeProcessor<TLeft, UInt8>::execute(type2);
if (typeid_cast<const DataTypeUInt16 *>(&*right)) return RightTypeProcessor<TLeft, UInt16>::execute(type2);
if (typeid_cast<const DataTypeUInt32 *>(&*right)) return RightTypeProcessor<TLeft, UInt32>::execute(type2);
if (typeid_cast<const DataTypeUInt64 *>(&*right)) return RightTypeProcessor<TLeft, UInt64>::execute(type2);
if (typeid_cast<const DataTypeInt8 *>(&*right)) return RightTypeProcessor<TLeft, Int8>::execute(type2);
if (typeid_cast<const DataTypeInt16 *>(&*right)) return RightTypeProcessor<TLeft, Int16>::execute(type2);
if (typeid_cast<const DataTypeInt32 *>(&*right)) return RightTypeProcessor<TLeft, Int32>::execute(type2);
if (typeid_cast<const DataTypeInt64 *>(&*right)) return RightTypeProcessor<TLeft, Int64>::execute(type2);
if (typeid_cast<const DataTypeFloat32 *>(&*right)) return RightTypeProcessor<TLeft, Float32>::execute(type2);
if (typeid_cast<const DataTypeFloat64 *>(&*right)) return RightTypeProcessor<TLeft, Float64>::execute(type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
};
return type_res;
}
template <typename T1>
DataTypePtr getSmallestCommonNumericTypeLeft(const IDataType & t2)
DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeTraits::EnrichedDataTypePtr & type1, const IDataType & type2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt8>();
if (typeid_cast<const DataTypeUInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt16>();
if (typeid_cast<const DataTypeUInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt32>();
if (typeid_cast<const DataTypeUInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, UInt64>();
if (typeid_cast<const DataTypeInt8 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int8>();
if (typeid_cast<const DataTypeInt16 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int16>();
if (typeid_cast<const DataTypeInt32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int32>();
if (typeid_cast<const DataTypeInt64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Int64>();
if (typeid_cast<const DataTypeFloat32 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float32>();
if (typeid_cast<const DataTypeFloat64 *>(&t2)) return getSmallestCommonNumericTypeImpl<T1, Float64>();
const DataTypePtr & left = type1.first;
const DataTypePtr & right = type1.second;
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
DataTypePtr getSmallestCommonNumericType(const IDataType & t1, const IDataType & t2)
{
if (typeid_cast<const DataTypeUInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt8>(t2);
if (typeid_cast<const DataTypeUInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt16>(t2);
if (typeid_cast<const DataTypeUInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt32>(t2);
if (typeid_cast<const DataTypeUInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<UInt64>(t2);
if (typeid_cast<const DataTypeInt8 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int8>(t2);
if (typeid_cast<const DataTypeInt16 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int16>(t2);
if (typeid_cast<const DataTypeInt32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int32>(t2);
if (typeid_cast<const DataTypeInt64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Int64>(t2);
if (typeid_cast<const DataTypeFloat32 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float32>(t2);
if (typeid_cast<const DataTypeFloat64 *>(&t1)) return getSmallestCommonNumericTypeLeft<Float64>(t2);
if (typeid_cast<const DataTypeUInt8 *>(&*left)) return LeftTypeProcessor<UInt8>::execute(right, type2);
if (typeid_cast<const DataTypeUInt16 *>(&*left)) return LeftTypeProcessor<UInt16>::execute(right, type2);
if (typeid_cast<const DataTypeUInt32 *>(&*left)) return LeftTypeProcessor<UInt32>::execute(right, type2);
if (typeid_cast<const DataTypeUInt64 *>(&*left)) return LeftTypeProcessor<UInt64>::execute(right, type2);
if (typeid_cast<const DataTypeInt8 *>(&*left)) return LeftTypeProcessor<Int8>::execute(right, type2);
if (typeid_cast<const DataTypeInt16 *>(&*left)) return LeftTypeProcessor<Int16>::execute(right, type2);
if (typeid_cast<const DataTypeInt32 *>(&*left)) return LeftTypeProcessor<Int32>::execute(right, type2);
if (typeid_cast<const DataTypeInt64 *>(&*left)) return LeftTypeProcessor<Int64>::execute(right, type2);
if (typeid_cast<const DataTypeFloat32 *>(&*left)) return LeftTypeProcessor<Float32>::execute(right, type2);
if (typeid_cast<const DataTypeFloat64 *>(&*left)) return LeftTypeProcessor<Float64>::execute(right, type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -44,6 +44,9 @@
#include <DB/Parsers/formatAST.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/Functions/FunctionsTransform.h>
#include <DB/Functions/FunctionsConditional.h>
#include <DB/Functions/FunctionsArray.h>
#include <ext/range.hpp>
#include <DB/DataTypes/DataTypeFactory.h>
@ -125,6 +128,37 @@ bool functionIsInOrGlobalInOperator(const String & name)
return name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn";
}
/// Create a function uniquely identified by the first two parameters. If this
/// function actually is a CASE expression, record this information.
FunctionPtr getFunctionFromFactory(const String & name, const ASTFunction::Genus genus, const Context & context)
{
FunctionPtr function = FunctionFactory::instance().get(name, context);
if (genus == ASTFunction::Genus::CASE_WITH_EXPR)
{
FunctionTransform * fun_transform = typeid_cast<FunctionTransform *>(&*function);
if (fun_transform == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
fun_transform->setCaseMode();
}
else if (genus == ASTFunction::Genus::CASE_WITHOUT_EXPR)
{
FunctionMultiIf * fun_multi_if = typeid_cast<FunctionMultiIf *>(&*function);
if (fun_multi_if == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
fun_multi_if->setCaseMode();
}
else if (genus == ASTFunction::Genus::CASE_ARRAY)
{
FunctionArray * fun_array = typeid_cast<FunctionArray *>(&*function);
if (fun_array == nullptr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
fun_array->setCaseMode();
}
return function;
}
}
void ExpressionAnalyzer::init()
@ -1622,7 +1656,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
return;
}
const FunctionPtr & function = FunctionFactory::instance().get(node->name, context);
const FunctionPtr & function = getFunctionFromFactory(node->name, node->genus, context);
Names argument_names;
DataTypes argument_types;

View File

@ -5,6 +5,62 @@
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_FUNCTION_GENUS;
}
String ASTFunction::getColumnName() const
{
String res;
WriteBufferFromString wb(res);
writeString(name, wb);
if (parameters)
{
writeChar('(', wb);
for (ASTs::const_iterator it = parameters->children.begin(); it != parameters->children.end(); ++it)
{
if (it != parameters->children.begin())
writeCString(", ", wb);
writeString((*it)->getColumnName(), wb);
}
writeChar(')', wb);
}
writeChar('(', wb);
for (ASTs::const_iterator it = arguments->children.begin(); it != arguments->children.end(); ++it)
{
if (it != arguments->children.begin())
writeCString(", ", wb);
writeString((*it)->getColumnName(), wb);
}
writeChar(')', wb);
return res;
}
/** Получить текст, который идентифицирует этот элемент. */
String ASTFunction::getID() const
{
return "Function_" + name;
}
ASTPtr ASTFunction::clone() const
{
ASTFunction * res = new ASTFunction(*this);
ASTPtr ptr{res};
res->children.clear();
if (arguments) { res->arguments = arguments->clone(); res->children.push_back(res->arguments); }
if (parameters) { res->parameters = parameters->clone(); res->children.push_back(res->parameters); }
return ptr;
}
void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
FormatStateStacked nested_need_parens = frame;
@ -12,6 +68,12 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
nested_need_parens.need_parens = true;
nested_dont_need_parens.need_parens = false;
if ((genus == Genus::CASE_WITH_EXPR) || (genus == Genus::CASE_WITHOUT_EXPR))
{
formatCase(settings, state, frame);
return;
}
/// Стоит ли записать эту функцию в виде оператора?
bool written = false;
if (arguments && !parameters)
@ -204,4 +266,81 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
}
}
void ASTFunction::formatCase(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
static constexpr auto s_case = "CASE";
static constexpr auto s_when = "WHEN";
static constexpr auto s_then = "THEN";
static constexpr auto s_else = "ELSE";
static constexpr auto s_end = "END";
static constexpr auto s_ws = " ";
const ASTExpressionList * expr_list = static_cast<const ASTExpressionList *>(&*arguments);
const ASTs & args = expr_list->children;
frame.need_parens = false;
std::string indent_str = settings.one_line ? "" : std::string(4 * frame.indent, ' ');
std::string indent_str2 = settings.one_line ? "" : std::string(4 * (frame.indent + 1), ' ');
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_case << s_ws
<< (settings.hilite ? hilite_none : "");
if (genus == Genus::CASE_WITH_EXPR)
{
args[0]->formatImpl(settings, state, frame);
settings.ostr << settings.nl_or_ws;
const ASTFunction * src_array_function = static_cast<const ASTFunction *>(&*args[1]);
const ASTExpressionList * src_expr_list = static_cast<const ASTExpressionList *>(&*src_array_function->arguments);
const ASTFunction * dst_array_function = static_cast<const ASTFunction *>(&*args[2]);
const ASTExpressionList * dst_expr_list = static_cast<const ASTExpressionList *>(&*dst_array_function->arguments);
size_t size = src_expr_list->children.size();
for (size_t i = 0; i < size; ++i)
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< indent_str2 << s_when << s_ws;
src_expr_list->children[i]->formatImpl(settings, state, frame);
settings.ostr << s_ws;
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_then << s_ws;
dst_expr_list->children[i]->formatImpl(settings, state, frame);
settings.ostr << settings.nl_or_ws;
}
}
else if (genus == Genus::CASE_WITHOUT_EXPR)
{
settings.ostr << settings.nl_or_ws;
for (size_t i = 0; i < (args.size() - 1); ++i)
{
if ((i % 2) == 0)
{
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< indent_str2 << s_when << s_ws;
args[i]->formatImpl(settings, state, frame);
settings.ostr << " ";
}
else
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << s_then << s_ws;
args[i]->formatImpl(settings, state, frame);
settings.ostr << settings.nl_or_ws;
}
}
}
else
throw Exception{"Invalid function genus", ErrorCodes::INVALID_FUNCTION_GENUS};
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str2
<< s_else << s_ws;
args.back()->formatImpl(settings, state, frame);
settings.ostr << settings.nl_or_ws;
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << s_end;
}
}

View File

@ -17,6 +17,7 @@
#include <DB/Parsers/CommonParsers.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/Parsers/ParserCase.h>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ParserCreateQuery.h>
@ -296,8 +297,10 @@ bool ParserCastExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & ma
const auto & id = typeid_cast<const ASTIdentifier &>(*identifier).name;
if (id.length() != strlen(name) || 0 != strcasecmp(id.c_str(), name))
/// Parse as a simple ASTFunction
return ParserFunction{}.parse(pos = begin, end, node, max_parsed_pos, expected);
{
/// Parse as a CASE expression.
return ParserCase{}.parse(pos = begin, end, node, max_parsed_pos, expected);
}
/// Parse as CAST(expression AS type)
ParserString open("("), close(")"), comma(",");

View File

@ -0,0 +1,164 @@
#include <DB/Parsers/ParserCase.h>
#include <DB/Parsers/ExpressionElementParsers.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/Parsers/ASTFunction.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Core/Field.h>
namespace DB
{
bool ParserCase::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
ParserWhiteSpaceOrComments ws;
ParserString s_case{"CASE", true, true};
ParserString s_when{"WHEN", true, true};
ParserString s_then{"THEN", true, true};
ParserString s_else{"ELSE", true, true};
ParserString s_end{ "END", true, true};
ParserExpressionWithOptionalAlias p_expr{false};
if (!s_case.parse(pos, end, node, max_parsed_pos, expected))
{
/// Parse as a simple ASTFunction.
return ParserFunction{}.parse(pos = begin, end, node, max_parsed_pos, expected);
}
ws.ignore(pos, end);
bool has_case_expr = false;
auto old_pos = pos;
has_case_expr = !s_when.parse(pos, end, node, max_parsed_pos, expected);
pos = old_pos;
ASTs args;
using ASTFunctionPtr = Poco::SharedPtr<ASTFunction>;
using ASTExpressionListPtr = Poco::SharedPtr<ASTExpressionList>;
auto parse_branches = [&]()
{
bool has_branch = false;
while (s_when.parse(pos, end, node, max_parsed_pos, expected))
{
has_branch = true;
ws.ignore(pos, end);
ASTPtr expr_when;
if (!p_expr.parse(pos, end, expr_when, max_parsed_pos, expected))
return false;
args.push_back(expr_when);
ws.ignore(pos, end);
if (!s_then.parse(pos, end, node, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
ASTPtr expr_then;
if (!p_expr.parse(pos, end, expr_then, max_parsed_pos, expected))
return false;
args.push_back(expr_then);
ws.ignore(pos, end);
}
if (!has_branch)
return false;
ws.ignore(pos, end);
if (!s_else.parse(pos, end, node, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
ASTPtr expr_else;
if (!p_expr.parse(pos, end, expr_else, max_parsed_pos, expected))
return false;
args.push_back(expr_else);
ws.ignore(pos, end);
if (!s_end.parse(pos, end, node, max_parsed_pos, expected))
return false;
return true;
};
if (has_case_expr)
{
ASTPtr case_expr;
if (!p_expr.parse(pos, end, case_expr, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
if (!parse_branches())
return false;
/// Hand-craft a transform() function.
ASTExpressionListPtr src_expr_list = new ASTExpressionList{StringRange{begin, pos}};
ASTExpressionListPtr dst_expr_list = new ASTExpressionList{StringRange{begin, pos}};
for (size_t i = 0; i < (args.size() - 1); ++i)
{
if ((i % 2) == 0)
src_expr_list->children.push_back(args[i]);
else
dst_expr_list->children.push_back(args[i]);
}
ASTFunctionPtr src_array_function = new ASTFunction{StringRange{begin, pos}};
src_array_function->name = "array";
src_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
src_array_function->arguments = src_expr_list;
src_array_function->children.push_back(src_array_function->arguments);
ASTFunctionPtr dst_array_function = new ASTFunction{StringRange{begin, pos}};
dst_array_function->name = "array";
dst_array_function->genus = ASTFunction::Genus::CASE_ARRAY;
dst_array_function->arguments = dst_expr_list;
dst_array_function->children.push_back(dst_array_function->arguments);
ASTPtr function_args = new ASTExpressionList{StringRange{begin, pos}};
function_args->children.push_back(case_expr);
function_args->children.push_back(src_array_function);
function_args->children.push_back(dst_array_function);
function_args->children.emplace_back(args.back());
ASTFunctionPtr function = new ASTFunction{StringRange{begin, pos}};
function->name = "transform";
function->genus = ASTFunction::Genus::CASE_WITH_EXPR;
function->arguments = function_args;
function->children.push_back(function->arguments);
node = function;
}
else
{
if (!parse_branches())
return false;
/// Hand-craft a multiIf() function.
ASTPtr function_args = new ASTExpressionList{StringRange{begin, pos}};
function_args->children = std::move(args);
ASTFunctionPtr function = new ASTFunction{StringRange{begin, pos}};
function->name = "multiIf";
function->genus = ASTFunction::Genus::CASE_WITHOUT_EXPR;
function->arguments = function_args;
function->children.push_back(function->arguments);
node = function;
}
return true;
}
}

View File

@ -1367,7 +1367,8 @@ SELECT multiIf((number % 2) = 0, [toFloat64(1), toFloat64(2)], (number % 3) = 0,
/* String branches */
CREATE TABLE IF NOT EXISTS test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String) ENGINE=TinyLog;
DROP TABLE IF EXISTS test.multi_if_check;
CREATE TABLE test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String) ENGINE=TinyLog;
INSERT INTO test.multi_if_check(col1, col2, col3, col4) VALUES(1, 'A', 'AB', 'ABC');
INSERT INTO test.multi_if_check(col1, col2, col3, col4) VALUES(2, 'B', 'BC', 'BCD');
INSERT INTO test.multi_if_check(col1, col2, col3, col4) VALUES(3, 'C', 'CD', 'CDE');
@ -1636,11 +1637,11 @@ SELECT multiIf(1, 'foo', 1, 'bar', toFixedString(col4, 16)) FROM test.multi_if_c
SELECT multiIf(1, 'foo', 1, 'bar', toFixedString('baz', 16)) FROM test.multi_if_check;
SELECT multiIf(1, 'foo', 1, 'bar', 'baz') FROM test.multi_if_check;
DROP TABLE test.multi_if_check;
DROP TABLE IF EXISTS test.multi_if_check;
/* String array branches */
CREATE TABLE IF NOT EXISTS test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=TinyLog;
CREATE TABLE test.multi_if_check(col1 UInt64, col2 String, col3 String, col4 String, col5 String, col6 String, col7 String) ENGINE=TinyLog;
INSERT INTO test.multi_if_check(col1, col2, col3, col4, col5, col6, col7) VALUES(1, 'A', 'AB', 'ABC', 'ABCD', 'ABCDE', 'ABCDEF');
INSERT INTO test.multi_if_check(col1, col2, col3, col4, col5, col6, col7) VALUES(2, 'B', 'BC', 'BCD', 'BCDE', 'BCDEF', 'BCDEFG');
INSERT INTO test.multi_if_check(col1, col2, col3, col4, col5, col6, col7) VALUES(3, 'C', 'CD', 'CDE', 'CDEF', 'CDEFG', 'CDEFGH');
@ -1909,4 +1910,4 @@ SELECT multiIf(1, ['foo', 'bar'], 1, ['foo', 'bar'], [col6, 'bar']) FROM test.mu
SELECT multiIf(1, ['foo', 'bar'], 1, ['foo', 'bar'], ['foo', col7]) FROM test.multi_if_check;
SELECT multiIf(1, ['foo', 'bar'], 1, ['foo', 'bar'], ['foo', 'bar']) FROM test.multi_if_check;
DROP TABLE test.multi_if_check;
DROP TABLE IF EXISTS test.multi_if_check;

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff