提交 2c1d7ad3 编写于 作者: A Alexey Milovidov

Simplification of Nullable [#CLICKHOUSE-2]

上级 6f53eb2b
......@@ -162,7 +162,7 @@ if (CMAKE_SYSTEM MATCHES "Linux" AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang")
option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX})
if (USE_LIBCXX)
set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -stdlib=libc++") # Ok for clang6, for older can cause 'not used option' warning
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=1") # More checks in debug build.
set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -D_LIBCPP_DEBUG=0") # More checks in debug build.
if (MAKE_STATIC_LIBRARIES)
link_libraries (-Wl,-Bstatic -stdlib=libc++ c++ c++abi -Wl,-Bdynamic)
else ()
......
......@@ -363,6 +363,7 @@ namespace ErrorCodes
extern const int CANNOT_APPLY_CATBOOST_MODEL = 383;
extern const int PART_IS_TEMPORARILY_LOCKED = 384;
extern const int MULTIPLE_STREAMS_REQUIRED = 385;
extern const int NO_COMMON_TYPE = 386;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;
......
......@@ -30,16 +30,11 @@ namespace ErrorCodes
DataTypeArray::DataTypeArray(const DataTypePtr & nested_)
: enriched_nested(std::make_pair(nested_, std::make_shared<DataTypeVoid>())), nested{nested_}
: nested{nested_}
{
offsets = std::make_shared<DataTypeNumber<ColumnArray::Offset_t>>();
}
DataTypeArray::DataTypeArray(const DataTypeTraits::EnrichedDataTypePtr & enriched_nested_)
: enriched_nested{enriched_nested_}, nested{enriched_nested.first}
{
offsets = std::make_shared<DataTypeNumber<ColumnArray::Offset_t>>();
}
void DataTypeArray::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
......
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/EnrichedDataTypePtr.h>
namespace DB
......@@ -11,8 +10,6 @@ namespace DB
class DataTypeArray final : public IDataType
{
private:
/// Extended type of array elements.
DataTypeTraits::EnrichedDataTypePtr enriched_nested;
/// The type of array elements.
DataTypePtr nested;
/// Type of offsets.
......@@ -22,7 +19,6 @@ public:
static constexpr bool is_parametric = true;
DataTypeArray(const DataTypePtr & nested_);
DataTypeArray(const DataTypeTraits::EnrichedDataTypePtr & enriched_nested_);
std::string getName() const override
{
......@@ -41,7 +37,7 @@ public:
DataTypePtr clone() const override
{
return std::make_shared<DataTypeArray>(enriched_nested);
return std::make_shared<DataTypeArray>(nested);
}
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
......@@ -95,7 +91,6 @@ public:
Field getDefault() const override;
const DataTypePtr & getNestedType() const { return nested; }
const DataTypeTraits::EnrichedDataTypePtr & getEnrichedNestedType() const { return enriched_nested; }
const DataTypePtr & getOffsetsType() const { return offsets; }
};
......
#include <sstream>
#include <Common/typeid_cast.h>
#include <DataTypes/getLeastCommonType.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int NO_COMMON_TYPE;
}
namespace
{
String getExceptionMessagePrefix(const DataTypes & types)
{
std::stringstream res;
res << "There is no common type for types ";
bool first = true;
for (const auto & type : types)
{
if (first)
res << ", ";
first = false;
res << type->getName();
}
return res.str();
}
}
DataTypePtr getLeastCommonType(const DataTypes & types)
{
/// Trivial cases
if (types.empty())
throw Exception("Empty list of types passed to getLeastCommonType function", ErrorCodes::BAD_ARGUMENTS);
if (types.size() == 1)
return types[0];
/// All types are equal
{
bool all_equal = true;
for (size_t i = 1, size = types.size(); i < size; ++i)
{
if (!types[i]->equals(*types[0]))
{
all_equal = false;
break;
}
}
if (all_equal)
return types[0];
}
/// Recursive rules
/// For Arrays
{
bool have_array = false;
bool all_arrays = true;
DataTypes nested_types;
nested_types.reserve(types.size());
for (const auto & type : types)
{
if (const DataTypeArray * type_array = typeid_cast<const DataTypeArray *>(type.get()))
{
have_array = true;
nested_types.emplace_back(type_array->getNestedType());
}
else
all_arrays = false;
}
if (have_array)
{
if (!all_arrays)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Array and some of them are not", ErrorCodes::NO_COMMON_TYPE);
return std::make_shared<DataTypeArray>(getLeastCommonType(nested_types));
}
}
/// For tuples
{
bool have_tuple = false;
bool all_tuples = true;
size_t tuple_size = 0;
std::vector<DataTypes> nested_types;
for (const auto & type : types)
{
if (const DataTypeTuple * type_tuple = typeid_cast<const DataTypeTuple *>(type.get()))
{
if (!have_tuple)
{
tuple_size = type_tuple->getElements().size();
nested_types.resize(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
nested_types[elem_idx].reserve(types.size());
}
else if (tuple_size != type_tuple->getElements().size())
throw Exception(getExceptionMessagePrefix(types) + " because Tuples have different sizes", ErrorCodes::NO_COMMON_TYPE);
have_tuple = true;
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
nested_types[elem_idx].emplace_back(type_tuple->getElements()[elem_idx]);
}
else
all_tuples = false;
if (have_tuple)
{
if (!all_tuples)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not", ErrorCodes::NO_COMMON_TYPE);
DataTypes common_tuple_types(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
common_tuple_types[elem_idx] = getLeastCommonType(nested_types[elem_idx]);
return std::make_shared<DataTypeTuple>(common_tuple_types);
}
}
}
/// For Nullable
{
bool have_nullable = false;
DataTypes nested_types;
nested_types.reserve(types.size());
for (const auto & type : types)
{
if (const DataTypeNullable * type_nullable = typeid_cast<const DataTypeNullable *>(type.get()))
{
have_nullable = true;
nested_types.emplace_back(type_nullable->getNestedType());
}
else
nested_types.emplace_back(type);
}
if (have_nullable)
{
return std::make_shared<DataTypeNullable>(getLeastCommonType(nested_types));
}
}
/// Non-recursive rules
/// For String and FixedString, or for different FixedStrings, the common type is String.
/// No other types are compatible with Strings.
{
bool have_string = false;
bool all_strings = true;
for (const auto & type : types)
{
if (typeid_cast<const DataTypeString *>(type.get())
|| typeid_cast<const DataTypeFixedString *>(type.get()))
have_string = true;
else
all_strings = false;
}
if (have_string)
{
if (!all_strings)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are String/FixedString and some of them are not", ErrorCodes::NO_COMMON_TYPE);
return std::make_shared<DataTypeString>();
}
}
/// For Date and DateTime, the common type is DateTime. No other types are compatible.
{
bool have_date_or_datetime = false;
bool all_date_or_datetime = true;
for (const auto & type : types)
{
if (typeid_cast<const DataTypeDate *>(type.get())
|| typeid_cast<const DataTypeDateTime *>(type.get()))
have_date_or_datetime = true;
else
all_date_or_datetime = false;
}
if (have_date_or_datetime)
{
if (!all_date_or_datetime)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Date/DateTime and some of them are not", ErrorCodes::NO_COMMON_TYPE);
return std::make_shared<DataTypeDateTime>();
}
}
/// For numeric types, the most complicated part.
{
bool all_numbers = true;
size_t max_bits_of_signed_integer = 0;
size_t max_bits_of_unsigned_integer = 0;
size_t max_mantissa_bits_of_floating = 0;
auto maximize = [](size_t & what, size_t value)
{
if (value > what)
what = value;
};
for (const auto & type : types)
{
if (typeid_cast<const DataTypeUInt8 *>(type.get()))
maximize(max_bits_of_unsigned_integer, 8);
else if (typeid_cast<const DataTypeUInt16 *>(type.get()))
maximize(max_bits_of_unsigned_integer, 16);
else if (typeid_cast<const DataTypeUInt32 *>(type.get()))
maximize(max_bits_of_unsigned_integer, 32);
else if (typeid_cast<const DataTypeUInt64 *>(type.get()))
maximize(max_bits_of_unsigned_integer, 64);
else if (typeid_cast<const DataTypeInt8 *>(type.get()))
maximize(max_bits_of_signed_integer, 8);
else if (typeid_cast<const DataTypeInt16 *>(type.get()))
maximize(max_bits_of_signed_integer, 16);
else if (typeid_cast<const DataTypeInt32 *>(type.get()))
maximize(max_bits_of_signed_integer, 32);
else if (typeid_cast<const DataTypeInt64 *>(type.get()))
maximize(max_bits_of_signed_integer, 64);
else if (typeid_cast<const DataTypeFloat32 *>(type.get()))
maximize(max_mantissa_bits_of_floating, 24);
else if (typeid_cast<const DataTypeFloat64 *>(type.get()))
maximize(max_mantissa_bits_of_floating, 53);
else
all_numbers = false;
}
if (max_bits_of_signed_integer || max_bits_of_unsigned_integer || max_mantissa_bits_of_floating)
{
if (!all_numbers)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are numbers and some of them are not", ErrorCodes::NO_COMMON_TYPE);
/// If there are signed and unsigned types of same bit-width, the result must be signed number with at least one more bit.
/// Example, common of Int32, UInt32 = Int64.
size_t min_bit_width_of_integer = std::max(max_bits_of_signed_integer, max_bits_of_unsigned_integer);
if (max_bits_of_signed_integer == max_bits_of_unsigned_integer)
++min_bit_width_of_integer;
/// If the result must be floating.
if (max_mantissa_bits_of_floating)
{
size_t min_mantissa_bits = std::max(min_bit_width_of_integer, max_mantissa_bits_of_floating);
if (min_mantissa_bits <= 24)
return std::make_shared<DataTypeFloat32>();
else if (min_mantissa_bits <= 53)
return std::make_shared<DataTypeFloat64>();
else
throw Exception(getExceptionMessagePrefix(types)
+ " because some of them are integers and some are floating point,"
" but there is no floating point type, that can exactly represent all required integers", ErrorCodes::NO_COMMON_TYPE);
}
/// If the result must be signed integer.
if (max_bits_of_signed_integer)
{
if (min_bit_width_of_integer <= 8)
return std::make_shared<DataTypeInt8>();
else if (min_bit_width_of_integer <= 16)
return std::make_shared<DataTypeInt16>();
else if (min_bit_width_of_integer <= 32)
return std::make_shared<DataTypeInt32>();
else if (min_bit_width_of_integer <= 64)
return std::make_shared<DataTypeInt64>();
else
throw Exception(getExceptionMessagePrefix(types)
+ " because some of them are signed integers and some are unsigned integers,"
" but there is no signed integer type, that can exactly represent all required unsigned integer values", ErrorCodes::NO_COMMON_TYPE);
}
/// All unsigned.
{
if (min_bit_width_of_integer <= 8)
return std::make_shared<DataTypeUInt8>();
else if (min_bit_width_of_integer <= 16)
return std::make_shared<DataTypeUInt16>();
else if (min_bit_width_of_integer <= 32)
return std::make_shared<DataTypeUInt32>();
else if (min_bit_width_of_integer <= 64)
return std::make_shared<DataTypeUInt64>();
else
throw Exception("Logical error: " + getExceptionMessagePrefix(types)
+ " but as all data types are unsigned integers, we must have found maximum unsigned integer type", ErrorCodes::NO_COMMON_TYPE);
}
}
}
/// All other data types (UUID, AggregateFunction, Enum...) are compatible only if they are the same (checked in trivial cases).
throw Exception(getExceptionMessagePrefix(types), ErrorCodes::NO_COMMON_TYPE);
}
}
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
/** Get data type that covers all possible values of passed data types.
* If there is no such data type, throws an exception.
*
* Examples: least common type for UInt8, Int8 - Int16.
* Examples: there is no common type for Array(UInt8), Int8.
*/
DataTypePtr getLeastCommonType(const DataTypes & types);
}
#include <Functions/Conditional/ArgsInfo.h>
#include <Functions/Conditional/CondException.h>
#include <Functions/Conditional/common.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/DataTypeTraits.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace Conditional
{
namespace
{
std::string dumpArgTypes(const DataTypes & args)
{
WriteBufferFromOwnString buf;
size_t else_arg = elseArg(args);
bool is_first = true;
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (is_first)
is_first = false;
else
writeString("; ", buf);
writeString(args[i]->getName(), buf);
}
writeString("; ", buf);
writeString(args[else_arg]->getName(), buf);
return buf.str();
}
/// Forward declarations.
template <typename TResult, typename TType>
class ResultDataTypeDeducer;
/// Internal class used by ResultDataTypeDeducer. Calls ResultDataTypeDeducer
/// for the next branch 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)
|| ResultDataTypeDeducer<TType, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeFloat32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeFloat64>>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
/// Specialization for the error type.
template <>
class ResultDataTypeDeducerImpl<NumberTraits::Error> final
{
public:
static void execute(const DataTypes & args, size_t, DataTypeTraits::EnrichedDataTypePtr &)
{
std::string dump = dumpArgTypes(args);
throw CondException{CondErrorCodes::TYPE_DEDUCER_UPSCALING_ERROR, dump};
}
};
template <typename TType>
struct TypeChecker
{
static bool execute(const DataTypePtr & arg)
{
if (arg->isNullable())
return false;
return typeid_cast<const TType *>(arg.get());
}
};
template <typename TType>
struct TypeChecker<Nullable<TType>>
{
static bool execute(const DataTypePtr & arg)
{
if (!arg->isNullable())
return false;
const DataTypeNullable & nullable_type = static_cast<DataTypeNullable &>(*arg);
const IDataType * nested_type = nullable_type.getNestedType().get();
return typeid_cast<const TType *>(nested_type);
}
};
/// 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 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 (!TypeChecker<TType>::execute(args[i]))
return false;
if (i == elseArg(args))
{
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);
}
return true;
}
};
/// 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 void execute(const DataTypes & args, DataTypeTraits::EnrichedDataTypePtr & type_res)
{
using Void = typename DataTypeTraits::ToEnrichedDataType<
NumberTraits::Enriched::Void<NumberTraits::HasNoNull>
>::Type;
size_t i = firstThen();
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)
|| ResultDataTypeDeducer<Void, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeFloat32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeFloat64>>::execute(args, i, type_res)))
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
}
};
}
DataTypePtr getReturnTypeForArithmeticArgs(const DataTypes & args)
{
DataTypeTraits::EnrichedDataTypePtr type_res;
FirstResultDataTypeDeducer::execute(args, type_res);
return type_res.first;
}
bool hasArithmeticBranches(const DataTypes & args)
{
auto check = [&args](size_t i)
{
return args[i]->behavesAsNumber() || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (!check(i))
return false;
}
return check(else_arg);
}
bool hasArrayBranches(const DataTypes & args)
{
auto check = [&args](size_t i)
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return checkDataType<DataTypeArray>(observed_type) || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))
return false;
}
return check(else_arg);
}
bool hasIdenticalTypes(const DataTypes & args)
{
auto check = [&args](size_t i, std::string & first_type_name)
{
if (!args[i]->isNull())
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
const std::string & name = observed_type->getName();
if (first_type_name.empty())
first_type_name = name;
else if (name != first_type_name)
return false;
}
return true;
};
size_t else_arg = elseArg(args);
std::string first_type_name;
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (!check(i, first_type_name))
return false;
}
return check(else_arg, first_type_name);
}
bool hasFixedStrings(const DataTypes & args)
{
auto check = [&args](size_t i)
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return checkDataType<DataTypeFixedString>(observed_type) || (args[i]->isNull());
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))
return false;
}
return check(else_arg);
}
bool hasFixedStringsOfIdenticalLength(const DataTypes & args)
{
auto check = [&args](size_t i, bool & has_length, size_t & first_length)
{
if (!args[i]->isNull())
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
/// Get the length of the fixed string currently being checked.
auto fixed_str = checkAndGetDataType<DataTypeFixedString>(observed_type);
if (!fixed_str)
throw CondException{CondErrorCodes::TYPE_DEDUCER_ILLEGAL_COLUMN_TYPE, toString(i)};
size_t length = fixed_str->getN();
if (!has_length)
{
has_length = true;
first_length = length;
}
else if (length != first_length)
return false;
}
return true;
};
size_t else_arg = elseArg(args);
bool has_length = false;
size_t first_length = 0;
for (size_t i = firstThen(); i < else_arg; i = nextThen(i))
{
if (!check(i, has_length, first_length))
return false;
}
return check(else_arg, has_length, first_length);
}
bool hasStrings(const DataTypes & args)
{
auto check = [&args](size_t i)
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
return checkDataType<DataTypeFixedString>(observed_type) ||
checkDataType<DataTypeString>(observed_type) || args[i]->isNull();
};
size_t else_arg = elseArg(args);
for (size_t i = firstThen(); i < elseArg(args); i = nextThen(i))
{
if (!check(i))
return false;
}
return check(else_arg);
}
}
}
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
namespace Conditional
{
/// Determine the return type of the function multiIf when all the
/// branches (then, else) have numeric types.
DataTypePtr getReturnTypeForArithmeticArgs(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) have numeric types.
bool hasArithmeticBranches(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are arrays.
bool hasArrayBranches(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) have the same type name.
bool hasIdenticalTypes(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are fixed strings.
bool hasFixedStrings(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are fixed strings of equal length.
bool hasFixedStringsOfIdenticalLength(const DataTypes & args);
/// Returns true if either all the branches are null or
/// all the non-null branches (then, else) are strings.
bool hasStrings(const DataTypes & args);
}
}
#include <Functions/Conditional/NumericPerformer.h>
#include <Functions/Conditional/NullMapBuilder.h>
#include <Functions/Conditional/ArgsInfo.h>
#include <Functions/Conditional/NumericEvaluator.h>
#include <Functions/Conditional/ArrayEvaluator.h>
#include <DataTypes/NumberTraits.h>
......
#include <Functions/Conditional/getArrayType.h>
#include <Functions/Conditional/CondException.h>
#include <Functions/Conditional/common.h>
#include <DataTypes/DataTypeTraits.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace Conditional
{
namespace
{
std::string dumpArgTypes(const DataTypes & args)
{
WriteBufferFromOwnString buf;
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);
}
return buf.str();
}
/// Forward declarations.
template <typename TResult, typename TType>
class 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)
|| ResultDataTypeDeducer<TType, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeUInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<DataTypeFloat32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<TType, Nullable<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};
}
};
template <typename TType>
struct TypeChecker
{
static bool execute(const DataTypePtr & arg)
{
if (arg->isNullable())
return false;
return typeid_cast<const TType *>(arg.get());
}
};
template <typename TType>
struct TypeChecker<Nullable<TType>>
{
static bool execute(const DataTypePtr & arg)
{
if (!arg->isNullable())
return false;
const DataTypeNullable & nullable_type = static_cast<DataTypeNullable &>(*arg);
const IDataType * nested_type = nullable_type.getNestedType().get();
return typeid_cast<const TType *>(nested_type);
}
};
/// 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 (!TypeChecker<TType>::execute(args[i]))
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<NumberTraits::HasNoNull>
>::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)
|| ResultDataTypeDeducer<Void, DataTypeNull>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeUInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt8>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt16>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeInt64>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<DataTypeFloat32>>::execute(args, i, type_res)
|| ResultDataTypeDeducer<Void, Nullable<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;
}
}
}
#pragma once
#include <DataTypes/IDataType.h>
#include <DataTypes/EnrichedDataTypePtr.h>
namespace DB
{
namespace Conditional
{
/// Determine the least common type of the elements of an array.
DataTypeTraits::EnrichedDataTypePtr getArrayType(const DataTypes & args);
}
}
......@@ -5,7 +5,7 @@
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsConversion.h>
#include <Functions/Conditional/getArrayType.h>
#include <DataTypes/getLeastCommonType.h>
#include <Functions/Conditional/CondException.h>
#include <Functions/GatherUtils.h>
#include <Common/HashTable/HashMap.h>
......@@ -42,319 +42,106 @@ FunctionArray::FunctionArray(const Context & context)
{
}
namespace
{
/// Is there at least one numeric argument among the specified ones?
bool foundNumericType(const DataTypes & args)
{
for (size_t i = 0; i < args.size(); ++i)
{
if (args[i]->behavesAsNumber())
return true;
else if (!args[i]->isNull())
return false;
}
return false;
}
/// Check if the specified arguments have the same type up to nullability or nullity.
bool hasArrayIdenticalTypes(const DataTypes & args)
{
std::string first_type_name;
for (size_t i = 0; i < args.size(); ++i)
{
if (!args[i]->isNull())
{
const IDataType * observed_type = DataTypeTraits::removeNullable(args[i]).get();
const std::string & name = observed_type->getName();
if (first_type_name.empty())
first_type_name = name;
else if (name != first_type_name)
return false;
}
}
return true;
}
/// Given a set, 'args', of types that have been deemed to be identical by the
/// function hasArrayIdenticalTypes(), deduce the element type of an array that
/// would be constructed from a set of values V, such that, for each i, the
/// type of V[i] is args[i].
DataTypePtr getArrayElementType(const DataTypes & args)
{
bool found_null = false;
bool found_nullable = false;
const DataTypePtr * ret = nullptr;
for (size_t i = 0; i < args.size(); ++i)
{
const auto & type = args[i];
if (type->isNull())
found_null = true;
else if (type->isNullable())
{
ret = &type;
found_nullable = true;
break;
}
else
ret = &type;
}
if (found_nullable)
return *ret;
else if (found_null)
{
if (ret)
return std::make_shared<DataTypeNullable>(*ret);
else
return std::make_shared<DataTypeNull>();
}
else
{
if (!ret)
throw Exception{"getArrayElementType: internal error", ErrorCodes::LOGICAL_ERROR};
else
return *ret;
}
}
template <typename T0, typename T1>
bool tryAddField(DataTypePtr type_res, const Field & f, Array & arr)
{
if (typeid_cast<const T0 *>(type_res.get()))
{
if (f.isNull())
arr.emplace_back();
else
arr.push_back(applyVisitor(FieldVisitorConvertToNumber<typename T1::FieldType>(), f));
return true;
}
return false;
}
}
bool FunctionArray::addField(DataTypePtr type_res, const Field & f, Array & arr) const
{
/// Otherwise, it is necessary
if ( tryAddField<DataTypeUInt8, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeUInt16, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeUInt32, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeUInt64, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeInt8, DataTypeInt64>(type_res, f, arr)
|| tryAddField<DataTypeInt16, DataTypeInt64>(type_res, f, arr)
|| tryAddField<DataTypeInt32, DataTypeInt64>(type_res, f, arr)
|| tryAddField<DataTypeInt64, DataTypeInt64>(type_res, f, arr)
|| tryAddField<DataTypeFloat32, DataTypeFloat64>(type_res, f, arr)
|| tryAddField<DataTypeFloat64, DataTypeFloat64>(type_res, f, arr) )
return true;
else
{
throw Exception{"Illegal result type " + type_res->getName() + " of function " + getName(),
ErrorCodes::LOGICAL_ERROR};
}
}
const DataTypePtr & FunctionArray::getScalarType(const DataTypePtr & type)
{
const auto array = checkAndGetDataType<DataTypeArray>(type.get());
if (!array)
return type;
return getScalarType(array->getNestedType());
}
DataTypeTraits::EnrichedDataTypePtr FunctionArray::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 (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;
}
DataTypePtr FunctionArray::getReturnTypeImpl(const DataTypes & arguments) const
{
if (arguments.empty())
throw Exception{"Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (foundNumericType(arguments))
{
/// Since we have found at least one numeric argument, we infer that all
/// the arguments are numeric up to nullity. Let's determine the least
/// common type.
auto enriched_result_type = getLeastCommonType(arguments);
return std::make_shared<DataTypeArray>(enriched_result_type);
}
else
{
/// Otherwise all the arguments must have the same type up to nullability or nullity.
if (!hasArrayIdenticalTypes(arguments))
throw Exception{"Arguments for function array must have same type or behave as number.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeArray>(getArrayElementType(arguments));
}
return std::make_shared<DataTypeArray>(getLeastCommonType(arguments));
}
void FunctionArray::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
{
size_t num_elements = arguments.size();
bool is_const = true;
for (const auto arg_num : arguments)
{
if (!block.getByPosition(arg_num).column->isConst())
{
is_const = false;
break;
}
}
const DataTypePtr & return_type = block.getByPosition(result).type;
const DataTypePtr & elem_type = static_cast<const DataTypeArray &>(*return_type).getNestedType();
if (is_const)
{
const DataTypePtr & observed_type = DataTypeTraits::removeNullable(elem_type);
size_t block_size = block.rows();
Array arr;
for (const auto arg_num : arguments)
{
const auto & elem = block.getByPosition(arg_num);
/** If part of columns have not same type as common type of all elements of array,
* then convert them to common type.
* If part of columns are constants,
* then convert them to full columns.
*/
if (DataTypeTraits::removeNullable(elem.type)->equals(*observed_type))
{
/// If an element of the same type as the result, just add it in response
arr.push_back((*elem.column)[0]);
}
else if (elem.type->isNull())
arr.emplace_back();
else
{
/// Otherwise, you need to cast it to the result type
addField(observed_type, (*elem.column)[0], arr);
}
}
Columns columns_holder(num_elements);
const IColumn * columns[num_elements];
const auto first_arg = block.getByPosition(arguments[0]);
block.getByPosition(result).column = return_type->createConstColumn(first_arg.column->size(), arr);
}
else
for (size_t i = 0; i < num_elements; ++i)
{
size_t block_size = block.rows();
/** If part of columns have not same type as common type of all elements of array,
* then convert them to common type.
* If part of columns are constants,
* then convert them to full columns.
*/
const auto & arg = block.getByPosition(arguments[i]);
Columns columns_holder(num_elements);
const IColumn * columns[num_elements];
String elem_type_name = elem_type->getName();
ColumnPtr preprocessed_column = arg.column;
for (size_t i = 0; i < num_elements; ++i)
if (arg.type->getName() != elem_type_name)
{
const auto & arg = block.getByPosition(arguments[i]);
String elem_type_name = elem_type->getName();
ColumnPtr preprocessed_column = arg.column;
if (arg.type->getName() != elem_type_name)
Block temporary_block
{
Block temporary_block
{
{
arg.column,
arg.type,
arg.name
},
{
DataTypeString().createConstColumn(block_size, elem_type_name),
std::make_shared<DataTypeString>(),
""
},
{
nullptr,
elem_type,
""
}
};
FunctionCast func_cast(context);
arg.column,
arg.type,
arg.name
},
{
DataTypePtr unused_return_type;
ColumnsWithTypeAndName arguments{ temporary_block.getByPosition(0), temporary_block.getByPosition(1) };
std::vector<ExpressionAction> unused_prerequisites;
/// Prepares function to execution. TODO It is not obvious.
func_cast.getReturnTypeAndPrerequisites(arguments, unused_return_type, unused_prerequisites);
DataTypeString().createConstColumn(block_size, elem_type_name),
std::make_shared<DataTypeString>(),
""
},
{
nullptr,
elem_type,
""
}
};
func_cast.execute(temporary_block, {0, 1}, 2);
preprocessed_column = temporary_block.getByPosition(2).column;
}
FunctionCast func_cast(context);
{
DataTypePtr unused_return_type;
ColumnsWithTypeAndName arguments{ temporary_block.getByPosition(0), temporary_block.getByPosition(1) };
std::vector<ExpressionAction> unused_prerequisites;
if (auto materialized_column = preprocessed_column->convertToFullColumnIfConst())
preprocessed_column = materialized_column;
/// Prepares function to execution. TODO It is not obvious.
func_cast.getReturnTypeAndPrerequisites(arguments, unused_return_type, unused_prerequisites);
}
columns_holder[i] = std::move(preprocessed_column);
columns[i] = columns_holder[i].get();
func_cast.execute(temporary_block, {0, 1}, 2);
preprocessed_column = temporary_block.getByPosition(2).column;
}
/** Create and fill the result array.
*/
if (auto materialized_column = preprocessed_column->convertToFullColumnIfConst())
preprocessed_column = materialized_column;
auto out = std::make_shared<ColumnArray>(elem_type->createColumn());
IColumn & out_data = out->getData();
IColumn::Offsets_t & out_offsets = out->getOffsets();
columns_holder[i] = std::move(preprocessed_column);
columns[i] = columns_holder[i].get();
}
out_data.reserve(block_size * num_elements);
out_offsets.resize(block_size);
/** Create and fill the result array.
*/
IColumn::Offset_t current_offset = 0;
for (size_t i = 0; i < block_size; ++i)
{
for (size_t j = 0; j < num_elements; ++j)
out_data.insertFrom(*columns[j], i);
auto out = std::make_shared<ColumnArray>(elem_type->createColumn());
IColumn & out_data = out->getData();
IColumn::Offsets_t & out_offsets = out->getOffsets();
current_offset += num_elements;
out_offsets[i] = current_offset;
}
out_data.reserve(block_size * num_elements);
out_offsets.resize(block_size);
block.getByPosition(result).column = out;
IColumn::Offset_t current_offset = 0;
for (size_t i = 0; i < block_size; ++i)
{
for (size_t j = 0; j < num_elements; ++j)
out_data.insertFrom(*columns[j], i);
current_offset += num_elements;
out_offsets[i] = current_offset;
}
block.getByPosition(result).column = out;
}
/// Implementation of FunctionArrayElement.
namespace ArrayImpl
......@@ -2893,37 +2680,12 @@ DataTypePtr FunctionArrayConcat::getReturnTypeImpl(const DataTypes & arguments)
if (arguments.empty())
throw Exception{"Function array requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
DataTypes nested_types;
nested_types.reserve(arguments.size());
for (size_t i : ext::range(0, arguments.size()))
{
auto argument = arguments[i].get();
if (auto data_type_array = checkAndGetDataType<DataTypeArray>(argument))
nested_types.push_back(data_type_array->getNestedType());
else
throw Exception(
"Argument " + toString(i) + " for function " + getName() + " must be an array but it has type "
+ argument->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (foundNumericType(nested_types))
{
/// Since we have found at least one numeric argument, we infer that all
/// the arguments are numeric up to nullity. Let's determine the least
/// common type.
auto enriched_result_type = Conditional::getArrayType(nested_types);
return std::make_shared<DataTypeArray>(enriched_result_type);
}
else
{
/// Otherwise all the arguments must have the same type up to nullability or nullity.
if (!hasArrayIdenticalTypes(nested_types))
throw Exception{"Arguments for function " + getName() + " must have same type or behave as number.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto array_type = typeid_cast<DataTypeArray *>(arguments[0].get());
if (!array_type)
throw Exception("First argument for function " + getName() + " must be an array but it has type "
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeArray>(getArrayElementType(nested_types));
}
return getLeastCommonType(arguments);
}
void FunctionArrayConcat::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
......@@ -3088,9 +2850,6 @@ void FunctionArraySlice::executeImpl(Block & block, const ColumnNumbers & argume
DataTypePtr FunctionArrayPush::getReturnTypeImpl(const DataTypes & arguments) const
{
if (arguments[0]->isNull())
return arguments[0];
auto array_type = typeid_cast<DataTypeArray *>(arguments[0].get());
if (!array_type)
throw Exception("First argument for function " + getName() + " must be an array but it has type "
......@@ -3100,23 +2859,7 @@ DataTypePtr FunctionArrayPush::getReturnTypeImpl(const DataTypes & arguments) co
DataTypes types = {nested_type, arguments[1]};
if (foundNumericType(types))
{
/// Since we have found at least one numeric argument, we infer that all
/// the arguments are numeric up to nullity. Let's determine the least
/// common type.
auto enriched_result_type = Conditional::getArrayType(types);
return std::make_shared<DataTypeArray>(enriched_result_type);
}
else
{
/// Otherwise all the arguments must have the same type up to nullability or nullity.
if (!hasArrayIdenticalTypes(types))
throw Exception{"Arguments for function " + getName() + " must have same type or behave as number.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeArray>(getArrayElementType(types));
}
return std::make_shared<DataTypeArray>(getLeastCommonType(types));
}
void FunctionArrayPush::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
......
......@@ -101,8 +101,6 @@ private:
String getName() const override;
bool addField(DataTypePtr type_res, const Field & f, Array & arr) const;
static const DataTypePtr & getScalarType(const DataTypePtr & type);
DataTypeTraits::EnrichedDataTypePtr getLeastCommonType(const DataTypes & arguments) const;
private:
const Context & context;
......
......@@ -4,7 +4,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/Conditional/common.h>
#include <Functions/Conditional/NullMapBuilder.h>
#include <Functions/Conditional/ArgsInfo.h>
#include <Functions/Conditional/CondSource.h>
#include <Functions/Conditional/NumericPerformer.h>
#include <Functions/Conditional/StringEvaluator.h>
......@@ -109,11 +108,6 @@ String FunctionMultiIf::getName() const
return name;
}
DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const
{
return getReturnTypeInternal(args);
}
void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, size_t result)
{
if (!blockHasSpecialBranches(block, args))
......@@ -152,127 +146,75 @@ void FunctionMultiIf::executeImpl(Block & block, const ColumnNumbers & args, siz
dest_col.column = std::make_shared<ColumnNullable>(source_col.column, builder.getNullMap());
}
DataTypePtr FunctionMultiIf::getReturnTypeInternal(const DataTypes & args) const
DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const
{
/// Arguments are the following: cond1, then1, cond2, then2, ... condN, thenN, else.
auto for_conditions = [&args](auto && f)
{
size_t conditions_end = args.size() - 1;
for (size_t i = 0; i < conditions_end; i += 2)
f(args[i]);
};
auto for_branches = [&args](auto && f)
{
size_t branches_end = args.size();
for (size_t i = 1; i < branches_end; i += 2)
f(args[i]);
f(args.back());
};
if (!Conditional::hasValidArgCount(args))
throw Exception{"Invalid number of arguments for function " + getName(),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
/// Check that conditions have valid types.
for (size_t i = Conditional::firstCond(); i < Conditional::elseArg(args); i = Conditional::nextCond(i))
/// Conditions must be UInt8, Nullable(UInt8) or Null. If one of conditions is Nullable, the result is also Nullable.
bool have_nullable_condition = false;
bool all_conditions_are_null = true;
for_conditions([&](const DataTypePtr & arg)
{
const IDataType * observed_type;
if (args[i]->isNullable())
if (arg->isNullable())
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
have_nullable_condition = true;
all_conditions_are_null = false;
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*arg);
observed_type = nullable_type.getNestedType().get();
}
else if (arg->isNull())
{
have_nullable_condition = true;
}
else
observed_type = args[i].get();
{
all_conditions_are_null = false;
observed_type = arg.get();
}
if (!checkDataType<DataTypeUInt8>(observed_type) && !observed_type->isNull())
throw Exception{"Illegal type of argument " + toString(i) + " (condition) "
"of function " + getName() + ". Must be UInt8.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
bool has_special_types = hasSpecialDataTypes(args);
if (Conditional::hasArithmeticBranches(args))
return Conditional::getReturnTypeForArithmeticArgs(args);
else if (Conditional::hasArrayBranches(args))
{
/// NOTE Error messages will refer to the types of array elements, which is slightly incorrect.
DataTypes new_args;
new_args.reserve(args.size());
});
auto push_branch_arg = [&args, &new_args](size_t i)
{
if (args[i]->isNull())
new_args.push_back(args[i]);
else
{
const IDataType * observed_type;
if (args[i]->isNullable())
{
const auto & nullable_type = static_cast<const DataTypeNullable &>(*args[i]);
observed_type = nullable_type.getNestedType().get();
}
else
observed_type = args[i].get();
const DataTypeArray * type_arr = checkAndGetDataType<DataTypeArray>(observed_type);
if (!type_arr)
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
new_args.push_back(type_arr->getNestedType());
}
};
for (size_t i = 0; i < Conditional::elseArg(args); ++i)
{
if (Conditional::isCond(i))
new_args.push_back(args[i]);
else
push_branch_arg(i);
}
if (all_conditions_are_null)
return std::make_shared<DataTypeNull>();
push_branch_arg(Conditional::elseArg(args));
DataTypes types_of_branches;
types_of_branches.reserve(Conditional::getBranchCount(args));
/// NOTE: in a future release, this code will be rewritten. Indeed
/// the current approach is flawed since it cannot appropriately
/// deal with null arguments and arrays that contain null elements.
/// For now we assume that arrays do not contain any such elements.
DataTypePtr elt_type = getReturnTypeImpl(new_args);
if (elt_type->isNullable())
{
DataTypeNullable & nullable_type = static_cast<DataTypeNullable &>(*elt_type);
elt_type = nullable_type.getNestedType();
}
DataTypePtr type = std::make_shared<DataTypeArray>(elt_type);
if (has_special_types)
type = std::make_shared<DataTypeNullable>(type);
return type;
}
else if (!Conditional::hasIdenticalTypes(args))
for_branches([&](const DataTypePtr & arg)
{
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};
auto type = getReturnTypeFromFirstNonNullBranch(args, has_special_types);
if (type)
return type;
else
{
/// This cannot happen: at least one fixed string is not null.
throw Exception{"Internal error", ErrorCodes::LOGICAL_ERROR};
}
}
else if (Conditional::hasStrings(args))
{
DataTypePtr type = std::make_shared<DataTypeString>();
if (has_special_types)
type = std::make_shared<DataTypeNullable>(type);
return type;
}
else
throw Exception{
"Incompatible branch (then, else) arguments for function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
else
{
auto type = getReturnTypeFromFirstNonNullBranch(args, has_special_types);
if (type)
return type;
types_of_branches.emplace_back(arg);
});
/// All the branches are null.
return std::make_shared<DataTypeNull>();
}
DataTypePtr common_type_of_branches = getLeastCommonType(types_of_branches);
return have_nullable_condition
? makeNullableDataTypeIfNot(common_type_of_branches)
: common_type_of_branches;
}
void FunctionMultiIf::perform(Block & block, const ColumnNumbers & args, size_t result, Conditional::NullMapBuilder & builder)
......
......@@ -1062,8 +1062,6 @@ public:
void executeImpl(Block & block, const ColumnNumbers & args, size_t result) override;
private:
DataTypePtr getReturnTypeInternal(const DataTypes & args) const;
/// Internal version of multiIf.
/// The builder parameter is an object that incrementally builds the null map
/// of the result column if it is nullable. When no builder is necessary,
......
......@@ -18,7 +18,6 @@
#include <Common/typeid_cast.h>
#include <common/StringRef.h>
#include <Common/HashTable/HashMap.h>
#include <DataTypes/EnrichedDataTypePtr.h>
namespace DB
{
......
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsTransform.h>
#include <DataTypes/DataTypeTraits.h>
namespace DB
{
namespace
{
template <typename TLeft, typename TRight, typename TType>
struct TypeProcessorImpl
{
static DataTypeTraits::EnrichedDataTypePtr execute()
{
using EnrichedT1 = std::tuple<TLeft, TRight, NumberTraits::HasNoNull>;
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 " + String(TypeName<TLeft>::get()) + " and " + String(TypeName<TType>::get())
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
}
};
template <typename TLeft, typename TRight>
struct RightTypeProcessor
{
static DataTypeTraits::EnrichedDataTypePtr execute(const IDataType & type2)
{
if (checkDataType<DataTypeUInt8>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt8>::execute();
if (checkDataType<DataTypeUInt16>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt16>::execute();
if (checkDataType<DataTypeUInt32>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt32>::execute();
if (checkDataType<DataTypeUInt64>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt64>::execute();
if (checkDataType<DataTypeInt8>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int8>::execute();
if (checkDataType<DataTypeInt16>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int16>::execute();
if (checkDataType<DataTypeInt32>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int32>::execute();
if (checkDataType<DataTypeInt64>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int64>::execute();
if (checkDataType<DataTypeFloat32>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float32>::execute();
if (checkDataType<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 TLeft>
struct LeftTypeProcessor
{
static DataTypeTraits::EnrichedDataTypePtr execute(const DataTypePtr & right, const IDataType & type2)
{
if (checkDataType<DataTypeVoid>(&*right)) return RightTypeProcessor<TLeft, void>::execute(type2);
if (checkDataType<DataTypeUInt8>(&*right)) return RightTypeProcessor<TLeft, UInt8>::execute(type2);
if (checkDataType<DataTypeUInt16>(&*right)) return RightTypeProcessor<TLeft, UInt16>::execute(type2);
if (checkDataType<DataTypeUInt32>(&*right)) return RightTypeProcessor<TLeft, UInt32>::execute(type2);
if (checkDataType<DataTypeUInt64>(&*right)) return RightTypeProcessor<TLeft, UInt64>::execute(type2);
if (checkDataType<DataTypeInt8>(&*right)) return RightTypeProcessor<TLeft, Int8>::execute(type2);
if (checkDataType<DataTypeInt16>(&*right)) return RightTypeProcessor<TLeft, Int16>::execute(type2);
if (checkDataType<DataTypeInt32>(&*right)) return RightTypeProcessor<TLeft, Int32>::execute(type2);
if (checkDataType<DataTypeInt64>(&*right)) return RightTypeProcessor<TLeft, Int64>::execute(type2);
if (checkDataType<DataTypeFloat32>(&*right)) return RightTypeProcessor<TLeft, Float32>::execute(type2);
if (checkDataType<DataTypeFloat64>(&*right)) return RightTypeProcessor<TLeft, Float64>::execute(type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
};
}
DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeTraits::EnrichedDataTypePtr & type1, const IDataType & type2)
{
const DataTypePtr & left = type1.first;
const DataTypePtr & right = type1.second;
if (checkDataType<DataTypeUInt8>(&*left)) return LeftTypeProcessor<UInt8>::execute(right, type2);
if (checkDataType<DataTypeUInt16>(&*left)) return LeftTypeProcessor<UInt16>::execute(right, type2);
if (checkDataType<DataTypeUInt32>(&*left)) return LeftTypeProcessor<UInt32>::execute(right, type2);
if (checkDataType<DataTypeUInt64>(&*left)) return LeftTypeProcessor<UInt64>::execute(right, type2);
if (checkDataType<DataTypeInt8>(&*left)) return LeftTypeProcessor<Int8>::execute(right, type2);
if (checkDataType<DataTypeInt16>(&*left)) return LeftTypeProcessor<Int16>::execute(right, type2);
if (checkDataType<DataTypeInt32>(&*left)) return LeftTypeProcessor<Int32>::execute(right, type2);
if (checkDataType<DataTypeInt64>(&*left)) return LeftTypeProcessor<Int64>::execute(right, type2);
if (checkDataType<DataTypeFloat32>(&*left)) return LeftTypeProcessor<Float32>::execute(right, type2);
if (checkDataType<DataTypeFloat64>(&*left)) return LeftTypeProcessor<Float64>::execute(right, type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
}
namespace DB
{
......
......@@ -15,7 +15,7 @@
#include <Common/typeid_cast.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <DataTypes/EnrichedDataTypePtr.h>
#include <DataTypes/getLeastCommonType.h>
namespace DB
......@@ -32,8 +32,6 @@ namespace ErrorCodes
/** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match.
*/
DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeTraits::EnrichedDataTypePtr & type1, const IDataType & type2);
/** transform(x, [from...], [to...], default)
* - converts the values according to the explicitly specified mapping.
*
......@@ -105,8 +103,7 @@ public:
throw Exception{"Third argument of function " + getName()
+ ", must be array of destination values to transform to.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
const auto enriched_type_arr_to_nested = type_arr_to->getEnrichedNestedType();
const auto & type_arr_to_nested = enriched_type_arr_to_nested.first;
const DataTypePtr & type_arr_to_nested = type_arr_to->getNestedType();
if (args_size == 3)
{
......@@ -136,8 +133,7 @@ public:
if (type_arr_to_nested->behavesAsNumber() && type_default->behavesAsNumber())
{
/// We take the smallest common type for the elements of the array of values `to` and for `default`.
DataTypeTraits::EnrichedDataTypePtr res = getSmallestCommonNumericType(enriched_type_arr_to_nested, *type_default);
return res.first;
return getLeastCommonType({type_arr_to_nested, type_default});
}
/// TODO More checks.
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册