#pragma once #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int CANNOT_PARSE_NUMBER; extern const int CANNOT_READ_ARRAY_FROM_TEXT; extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; extern const int CANNOT_PARSE_QUOTED_STRING; extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; extern const int CANNOT_PARSE_DATE; extern const int CANNOT_PARSE_DATETIME; extern const int CANNOT_PARSE_TEXT; extern const int CANNOT_PARSE_UUID; extern const int TOO_LARGE_STRING_SIZE; extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; } /** Type conversion functions. * toType - conversion in "natural way"; */ /** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment. * (Date is represented internally as number of days from some day; DateTime - as unix timestamp) */ template struct ConvertImpl { using FromFieldType = typename FromDataType::FieldType; using ToFieldType = typename ToDataType::FieldType; static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { if (const ColumnVector * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnVector::create(); const typename ColumnVector::Container & vec_from = col_from->getData(); typename ColumnVector::Container & vec_to = col_to->getData(); size_t size = vec_from.size(); vec_to.resize(size); for (size_t i = 0; i < size; ++i) vec_to[i] = static_cast(vec_from[i]); block.getByPosition(result).column = std::move(col_to); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } }; /** Conversion of Date to DateTime: adding 00:00:00 time component. */ struct ToDateTimeImpl { static constexpr auto name = "toDateTime"; static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone) { return time_zone.fromDayNum(DayNum_t(d)); } }; template struct ConvertImpl : DateTimeTransformImpl {}; /// Implementation of toDate function. template struct ToDateTransform32Or64 { static constexpr auto name = "toDate"; static inline ToType execute(const FromType & from, const DateLUTImpl & time_zone) { return (from < 0xFFFF) ? from : time_zone.toDayNum(from); } }; /** Conversion of DateTime to Date: throw off time component. */ template struct ConvertImpl : DateTimeTransformImpl {}; /** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date. * If number is less than 65536, then it is treated as DayNum, and if greater or equals, then as unix timestamp. * It's a bit illogical, as we actually have two functions in one. * But allows to support frequent case, * when user write toDate(UInt32), expecting conversion of unix timestamp to Date. * (otherwise such usage would be frequent mistake). */ template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; template struct ConvertImpl : DateTimeTransformImpl> {}; /** Transformation of numbers, dates, datetimes to strings: through formatting. */ template struct FormatImpl { static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType *, const DateLUTImpl *) { writeText(x, wb); } }; template <> struct FormatImpl { static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *) { writeDateText(DayNum_t(x), wb); } }; template <> struct FormatImpl { static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime *, const DateLUTImpl * time_zone) { writeDateTimeText(x, wb, *time_zone); } }; template struct FormatImpl> { static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum * type, const DateLUTImpl *) { writeString(type->getNameForValue(x), wb); } }; /// DataTypeEnum to DataType free conversion template struct ConvertImpl, DataTypeNumber, Name> { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { block.getByPosition(result).column = block.getByPosition(arguments[0]).column; } }; template struct ConvertImpl, DataTypeString>, Name> { using FromFieldType = typename FromDataType::FieldType; static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const auto & col_with_type_and_name = block.getByPosition(arguments[0]); const auto & type = static_cast(*col_with_type_and_name.type); const DateLUTImpl * time_zone = nullptr; /// For argument of DateTime type, second argument with time zone could be specified. if constexpr (std::is_same_v) time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0); if (const auto col_from = checkAndGetColumn>(col_with_type_and_name.column.get())) { auto col_to = ColumnString::create(); const typename ColumnVector::Container & vec_from = col_from->getData(); ColumnString::Chars_t & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = vec_from.size(); if constexpr (std::is_same_v) data_to.resize(size * (strlen("YYYY-MM-DD") + 1)); else if constexpr (std::is_same_v) data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1)); else data_to.resize(size * 3); /// Arbitary offsets_to.resize(size); WriteBufferFromVector write_buffer(data_to); for (size_t i = 0; i < size; ++i) { FormatImpl::execute(vec_from[i], write_buffer, &type, time_zone); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } data_to.resize(write_buffer.count()); block.getByPosition(result).column = std::move(col_to); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } }; /// Generic conversion of any type to String. struct ConvertImplGenericToString { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const auto & col_with_type_and_name = block.getByPosition(arguments[0]); const IDataType & type = *col_with_type_and_name.type; const IColumn & col_from = *col_with_type_and_name.column; size_t size = col_from.size(); auto col_to = ColumnString::create(); ColumnString::Chars_t & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitary. offsets_to.resize(size); WriteBufferFromVector write_buffer(data_to); for (size_t i = 0; i < size; ++i) { type.serializeText(col_from, i, write_buffer); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); } data_to.resize(write_buffer.count()); block.getByPosition(result).column = std::move(col_to); } }; /** Conversion of strings to numbers, dates, datetimes: through parsing. */ template void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { readText(x, rb); } template <> inline void parseImpl(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { DayNum_t tmp(0); readDateText(tmp, rb); x = tmp; } template <> inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { time_t tmp = 0; readDateTimeText(tmp, rb, *time_zone); x = tmp; } template <> inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { UUID tmp; readText(tmp, rb); x = tmp; } template bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb) { if constexpr (std::is_integral_v) return tryReadIntText(x, rb); else if constexpr (std::is_floating_point_v) return tryReadFloatText(x, rb); /// NOTE Need to implement for Date and DateTime too. } /** Throw exception with verbose message when string value is not parsed completely. */ void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result); enum class ConvertFromStringExceptionMode { Throw, /// Throw exception if value cannot be parsed. Zero, /// Fill with zero or default if value cannot be parsed. Null /// Return ColumnNullable with NULLs when value cannot be parsed. }; template struct ConvertThroughParsing { static_assert(std::is_same_v || std::is_same_v, "ConvertThroughParsing is only applicable for String or FixedString data tyoes"); using ToFieldType = typename ToDataType::FieldType; static bool isAllRead(ReadBuffer & in) { /// In case of FixedString, skip zero bytes at end. if constexpr (std::is_same_v) while (!in.eof() && *in.position() == 0) ++in.position(); if (in.eof()) return true; /// Special case, that allows to parse string with DateTime as Date. if (std::is_same_v && (in.buffer().size()) == strlen("YYYY-MM-DD hh:mm:ss")) return true; return false; } static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const DateLUTImpl * time_zone [[maybe_unused]] = nullptr; /// For conversion to DateTime type, second argument with time zone could be specified. if (std::is_same_v) time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0); const IColumn * col_from = block.getByPosition(arguments[0]).column.get(); const ColumnString * col_from_string = checkAndGetColumn(col_from); const ColumnFixedString * col_from_fixed_string = checkAndGetColumn(col_from); if (std::is_same_v && !col_from_string) throw Exception("Illegal column " + col_from->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); if (std::is_same_v && !col_from_fixed_string) throw Exception("Illegal column " + col_from->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); size_t size = block.rows(); auto col_to = ColumnVector::create(size); typename ColumnVector::Container & vec_to = col_to->getData(); ColumnUInt8::MutablePtr col_null_map_to; NullMap * vec_null_map_to = nullptr; if constexpr (mode == ConvertFromStringExceptionMode::Null) { col_null_map_to = ColumnUInt8::create(size); vec_null_map_to = &col_null_map_to->getData(); } const ColumnString::Chars_t * chars = nullptr; const IColumn::Offsets * offsets = nullptr; size_t fixed_string_size = 0; if constexpr (std::is_same_v) { chars = &col_from_string->getChars(); offsets = &col_from_string->getOffsets(); } else { chars = &col_from_fixed_string->getChars(); fixed_string_size = col_from_fixed_string->getN(); } size_t current_offset = 0; for (size_t i = 0; i < size; ++i) { size_t next_offset = std::is_same_v ? (*offsets)[i] : (current_offset + fixed_string_size); size_t string_size = std::is_same_v ? next_offset - current_offset - 1 : fixed_string_size; ReadBufferFromMemory read_buffer(&(*chars)[current_offset], string_size); if constexpr (mode == ConvertFromStringExceptionMode::Throw) { parseImpl(vec_to[i], read_buffer, time_zone); if (!isAllRead(read_buffer)) throwExceptionForIncompletelyParsedValue(read_buffer, block, result); } else { if (!tryParseImpl(vec_to[i], read_buffer) || !isAllRead(read_buffer)) { vec_to[i] = 0; if constexpr (mode == ConvertFromStringExceptionMode::Null) vec_null_map_to[i] = 1; } else if constexpr (mode == ConvertFromStringExceptionMode::Null) vec_null_map_to[i] = 0; } current_offset = next_offset; } if constexpr (mode == ConvertFromStringExceptionMode::Null) block.getByPosition(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to)); else block.getByPosition(result).column = std::move(col_to); } }; template struct ConvertImpl, DataTypeString>, ToDataType, Name> : ConvertThroughParsing {}; template struct ConvertImpl, DataTypeFixedString>, ToDataType, Name> : ConvertThroughParsing {}; /** Conversion from String through parsing, which returns default value instead of throwing an exception. */ template struct ConvertOrZeroImpl : ConvertThroughParsing {}; template struct ConvertOrNullImpl : ConvertThroughParsing {}; /// Generic conversion of any type from String. Used for complex types: Array and Tuple. struct ConvertImplGenericFromString { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { const IColumn & col_from = *block.getByPosition(arguments[0]).column; size_t size = col_from.size(); const IDataType & data_type_to = *block.getByPosition(result).type; if (const ColumnString * col_from_string = checkAndGetColumn(&col_from)) { auto res = data_type_to.createColumn(); if (!size) return; IColumn & column_to = *res; column_to.reserve(size); const ColumnString::Chars_t & chars = col_from_string->getChars(); const IColumn::Offsets & offsets = col_from_string->getOffsets(); size_t current_offset = 0; for (size_t i = 0; i < size; ++i) { ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1); data_type_to.deserializeTextEscaped(column_to, read_buffer); if (!read_buffer.eof()) throwExceptionForIncompletelyParsedValue(read_buffer, block, result); current_offset = offsets[i]; } block.getByPosition(result).column = std::move(res); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of conversion function from string", ErrorCodes::ILLEGAL_COLUMN); } }; /// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type. struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; }; template <> struct ConvertImpl : ConvertImpl {}; /** If types are identical, just take reference to column. */ template struct ConvertImpl, T, Name> { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { block.getByPosition(result).column = block.getByPosition(arguments[0]).column; } }; /** Conversion from FixedString to String. * Cutting sequences of zero bytes from end of strings. */ template struct ConvertImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { if (const ColumnFixedString * col_from = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnString::create(); const ColumnFixedString::Chars_t & data_from = col_from->getChars(); ColumnString::Chars_t & data_to = col_to->getChars(); ColumnString::Offsets & offsets_to = col_to->getOffsets(); size_t size = col_from->size(); size_t n = col_from->getN(); data_to.resize(size * (n + 1)); /// + 1 - zero terminator offsets_to.resize(size); size_t offset_from = 0; size_t offset_to = 0; for (size_t i = 0; i < size; ++i) { size_t bytes_to_copy = n; while (bytes_to_copy > 0 && data_from[offset_from + bytes_to_copy - 1] == 0) --bytes_to_copy; memcpy(&data_to[offset_to], &data_from[offset_from], bytes_to_copy); offset_from += n; offset_to += bytes_to_copy; data_to[offset_to] = 0; ++offset_to; offsets_to[i] = offset_to; } data_to.resize(offset_to); block.getByPosition(result).column = std::move(col_to); } else throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of first argument of function " + Name::name, ErrorCodes::ILLEGAL_COLUMN); } }; /// Declared early because used below. struct NameToDate { static constexpr auto name = "toDate"; }; struct NameToDateTime { static constexpr auto name = "toDateTime"; }; struct NameToString { static constexpr auto name = "toString"; }; #define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \ struct NameToInterval ## INTERVAL_KIND \ { \ static constexpr auto name = "toInterval" #INTERVAL_KIND; \ static constexpr int kind = DataTypeInterval::INTERVAL_KIND; \ }; DEFINE_NAME_TO_INTERVAL(Second) DEFINE_NAME_TO_INTERVAL(Minute) DEFINE_NAME_TO_INTERVAL(Hour) DEFINE_NAME_TO_INTERVAL(Day) DEFINE_NAME_TO_INTERVAL(Week) DEFINE_NAME_TO_INTERVAL(Month) DEFINE_NAME_TO_INTERVAL(Year) #undef DEFINE_NAME_TO_INTERVAL template class FunctionConvert : public IFunction { public: using Monotonic = MonotonicityImpl; static constexpr auto name = Name::name; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } bool isInjective(const Block &) override { return std::is_same_v; } void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector &) override { if constexpr (std::is_same_v) { out_return_type = std::make_shared(DataTypeInterval::Kind(Name::kind)); } else { /** Optional second argument with time zone is supported: * - for functions toDateTime, toUnixTimestamp, toDate; * - for function toString of DateTime argument. */ if (arguments.size() == 2) { if (!checkAndGetDataType(arguments[1].type.get())) throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (!(std::is_same_v || std::is_same_v || std::is_same_v || (std::is_same_v && checkDataType(arguments[0].type.get())))) { throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 1.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } if (std::is_same_v) out_return_type = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0)); else out_return_type = std::make_shared(); } } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { try { executeInternal(block, arguments, result); } catch (Exception & e) { /// More convenient error message. if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) { e.addMessage("Cannot parse " + block.getByPosition(result).type->getName() + " from " + block.getByPosition(arguments[0]).type->getName() + ", because value is too short"); } else if (e.code() == ErrorCodes::CANNOT_PARSE_NUMBER || e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT || e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING || e.code() == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE || e.code() == ErrorCodes::CANNOT_PARSE_DATE || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME || e.code() == ErrorCodes::CANNOT_PARSE_UUID) { e.addMessage("Cannot parse " + block.getByPosition(result).type->getName() + " from " + block.getByPosition(arguments[0]).type->getName()); } throw; } } bool hasInformationAboutMonotonicity() const override { return Monotonic::has(); } Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { return Monotonic::get(type, left, right); } private: void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result) { if (!arguments.size()) throw Exception{"Function " + getName() + " expects at least 1 arguments", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION}; const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else if (checkDataType(from_type)) ConvertImpl::execute(block, arguments, result); else { /// Generic conversion of any type to String. if (std::is_same_v) { ConvertImplGenericToString::execute(block, arguments, result); } else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } } }; /** Functions toTOrZero (where T is number of date or datetime type): * try to convert from String to type T through parsing, * if cannot parse, return default value instead of throwing exception. * NOTE Also need implement tryToUnixTimestamp with timezone. */ template class FunctionConvertOrZero : public IFunction { public: static constexpr auto name = Name::name; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } size_t getNumberOfArguments() const override { return 1; } DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } bool useDefaultImplementationForConstants() const override { return true; } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { const IDataType * from_type = block.getByPosition(arguments[0]).type.get(); if (checkAndGetDataType(from_type)) ConvertOrZeroImpl::execute(block, arguments, result); else throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName() + ". Only String argument is accepted for try-conversion function. For other arguments, use function without 'try'.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } }; /** Conversion to fixed string is implemented only for strings. */ class FunctionToFixedString : public IFunction { public: static constexpr auto name = "toFixedString"; static FunctionPtr create(const Context &) { return std::make_shared(); }; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 2; } bool isInjective(const Block &) override { return true; } void getReturnTypeAndPrerequisitesImpl(const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector & /*out_prerequisites*/) override { if (!arguments[1].column) throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN); if (!arguments[0].type->isStringOrFixedString()) throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED); const size_t n = getSize(arguments[1]); out_return_type = std::make_shared(n); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto n = getSize(block.getByPosition(arguments[1])); return execute(block, arguments, result, n); } static void execute(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n) { const auto & column = block.getByPosition(arguments[0]).column; if (const auto column_string = checkAndGetColumn(column.get())) { auto column_fixed = ColumnFixedString::create(n); auto & out_chars = column_fixed->getChars(); const auto & in_chars = column_string->getChars(); const auto & in_offsets = column_string->getOffsets(); out_chars.resize_fill(in_offsets.size() * n); for (size_t i = 0; i < in_offsets.size(); ++i) { const size_t off = i ? in_offsets[i - 1] : 0; const size_t len = in_offsets[i] - off - 1; if (len > n) throw Exception("String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE); memcpy(&out_chars[i * n], &in_chars[off], len); } block.getByPosition(result).column = std::move(column_fixed); } else if (const auto column_fixed_string = checkAndGetColumn(column.get())) { const auto src_n = column_fixed_string->getN(); if (src_n > n) throw Exception{ "String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE}; auto column_fixed = ColumnFixedString::create(n); auto & out_chars = column_fixed->getChars(); const auto & in_chars = column_fixed_string->getChars(); const auto size = column_fixed_string->size(); out_chars.resize_fill(size * n); for (const auto i : ext::range(0, size)) memcpy(&out_chars[i * n], &in_chars[i * src_n], src_n); block.getByPosition(result).column = std::move(column_fixed); } else throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN); } private: template bool getSizeTyped(const ColumnWithTypeAndName & column, size_t & out_size) { if (!checkDataType>(column.type.get())) return false; auto column_const = checkAndGetColumnConst>(column.column.get()); if (!column_const) throw Exception("Unexpected type of column for FixedString length: " + column.column->getName(), ErrorCodes::ILLEGAL_COLUMN); T s = column_const->template getValue(); if (s <= 0) throw Exception("FixedString length must be positive (unlike " + toString(s) + ")", ErrorCodes::ILLEGAL_COLUMN); out_size = static_cast(s); return true; } size_t getSize(const ColumnWithTypeAndName & column) { size_t res; if (getSizeTyped(column, res) || getSizeTyped(column, res) || getSizeTyped(column, res) || getSizeTyped(column, res) || getSizeTyped< Int8 >(column, res) || getSizeTyped< Int16>(column, res) || getSizeTyped< Int32>(column, res) || getSizeTyped< Int64>(column, res)) return res; throw Exception("Length of FixedString must be integer; got " + column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } }; /// Monotonicity. struct PositiveMonotonicity { static bool has() { return true; } static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &) { return { true }; } }; template struct ToIntMonotonicity { static bool has() { return true; } static UInt64 divideByRangeOfType(UInt64 x) { if constexpr (sizeof(T) < sizeof(UInt64)) return x >> (sizeof(T) * 8); else return 0; } static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { size_t size_of_type = type.getSizeOfValueInMemory(); /// If type is expanding, then function is monotonic. if (sizeof(T) > size_of_type) return { true, true, true }; /// If type is same, too. (Enum has separate case, because it is different data type) if (checkDataType>(&type) || checkDataType>(&type)) return { true, true, true }; /// In other cases, if range is unbounded, we don't know, whether function is monotonic or not. if (left.isNull() || right.isNull()) return {}; /// If converting from float, for monotonicity, arguments must fit in range of result type. if (checkDataType(&type) || checkDataType(&type)) { Float64 left_float = left.get(); Float64 right_float = right.get(); if (left_float >= std::numeric_limits::min() && left_float <= std::numeric_limits::max() && right_float >= std::numeric_limits::min() && right_float <= std::numeric_limits::max()) return { true }; return {}; } /// If signedness of type is changing, or converting from Date, DateTime, then arguments must be from same half, /// and after conversion, resulting values must be from same half. /// Just in case, it is required in rest of cases too. if ((left.get() >= 0) != (right.get() >= 0) || (T(left.get()) >= 0) != (T(right.get()) >= 0)) return {}; /// If type is shrinked, then for monotonicity, all bits other than that fits, must be same. if (divideByRangeOfType(left.get()) != divideByRangeOfType(right.get())) return {}; return { true }; } }; /** The monotonicity for the `toString` function is mainly determined for test purposes. * It is doubtful that anyone is looking to optimize queries with conditions `toString(CounterID) = 34`. */ struct ToStringMonotonicity { static bool has() { return true; } static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right) { IFunction::Monotonicity positive(true, true); IFunction::Monotonicity not_monotonic; /// `toString` function is monotonous if the argument is Date or DateTime, or non-negative numbers with the same number of symbols. if (checkAndGetDataType(&type) || typeid_cast(&type)) return positive; if (left.isNull() || right.isNull()) return {}; if (left.getType() == Field::Types::UInt64 && right.getType() == Field::Types::UInt64) { return (left.get() == 0 && right.get() == 0) || (floor(log10(left.get())) == floor(log10(right.get()))) ? positive : not_monotonic; } if (left.getType() == Field::Types::Int64 && right.getType() == Field::Types::Int64) { return (left.get() == 0 && right.get() == 0) || (left.get() > 0 && right.get() > 0 && floor(log10(left.get())) == floor(log10(right.get()))) ? positive : not_monotonic; } return not_monotonic; } }; struct NameToUInt8 { static constexpr auto name = "toUInt8"; }; struct NameToUInt16 { static constexpr auto name = "toUInt16"; }; struct NameToUInt32 { static constexpr auto name = "toUInt32"; }; struct NameToUInt64 { static constexpr auto name = "toUInt64"; }; struct NameToInt8 { static constexpr auto name = "toInt8"; }; struct NameToInt16 { static constexpr auto name = "toInt16"; }; struct NameToInt32 { static constexpr auto name = "toInt32"; }; struct NameToInt64 { static constexpr auto name = "toInt64"; }; struct NameToFloat32 { static constexpr auto name = "toFloat32"; }; struct NameToFloat64 { static constexpr auto name = "toFloat64"; }; struct NameToUUID { static constexpr auto name = "toUUID"; }; using FunctionToUInt8 = FunctionConvert>; using FunctionToUInt16 = FunctionConvert>; using FunctionToUInt32 = FunctionConvert>; using FunctionToUInt64 = FunctionConvert>; using FunctionToInt8 = FunctionConvert>; using FunctionToInt16 = FunctionConvert>; using FunctionToInt32 = FunctionConvert>; using FunctionToInt64 = FunctionConvert>; using FunctionToFloat32 = FunctionConvert; using FunctionToFloat64 = FunctionConvert; using FunctionToDate = FunctionConvert>; using FunctionToDateTime = FunctionConvert>; using FunctionToUUID = FunctionConvert>; using FunctionToString = FunctionConvert; using FunctionToUnixTimestamp = FunctionConvert>; template struct FunctionTo; template <> struct FunctionTo { using Type = FunctionToUInt8; }; template <> struct FunctionTo { using Type = FunctionToUInt16; }; template <> struct FunctionTo { using Type = FunctionToUInt32; }; template <> struct FunctionTo { using Type = FunctionToUInt64; }; template <> struct FunctionTo { using Type = FunctionToInt8; }; template <> struct FunctionTo { using Type = FunctionToInt16; }; template <> struct FunctionTo { using Type = FunctionToInt32; }; template <> struct FunctionTo { using Type = FunctionToInt64; }; template <> struct FunctionTo { using Type = FunctionToFloat32; }; template <> struct FunctionTo { using Type = FunctionToFloat64; }; template <> struct FunctionTo { using Type = FunctionToDate; }; template <> struct FunctionTo { using Type = FunctionToDateTime; }; template <> struct FunctionTo { using Type = FunctionToUUID; }; template <> struct FunctionTo { using Type = FunctionToString; }; template <> struct FunctionTo { using Type = FunctionToFixedString; }; template struct FunctionTo> : FunctionTo> { }; struct NameToUInt8OrZero { static constexpr auto name = "toUInt8OrZero"; }; struct NameToUInt16OrZero { static constexpr auto name = "toUInt16OrZero"; }; struct NameToUInt32OrZero { static constexpr auto name = "toUInt32OrZero"; }; struct NameToUInt64OrZero { static constexpr auto name = "toUInt64OrZero"; }; struct NameToInt8OrZero { static constexpr auto name = "toInt8OrZero"; }; struct NameToInt16OrZero { static constexpr auto name = "toInt16OrZero"; }; struct NameToInt32OrZero { static constexpr auto name = "toInt32OrZero"; }; struct NameToInt64OrZero { static constexpr auto name = "toInt64OrZero"; }; struct NameToFloat32OrZero { static constexpr auto name = "toFloat32OrZero"; }; struct NameToFloat64OrZero { static constexpr auto name = "toFloat64OrZero"; }; using FunctionToUInt8OrZero = FunctionConvertOrZero; using FunctionToUInt16OrZero = FunctionConvertOrZero; using FunctionToUInt32OrZero = FunctionConvertOrZero; using FunctionToUInt64OrZero = FunctionConvertOrZero; using FunctionToInt8OrZero = FunctionConvertOrZero; using FunctionToInt16OrZero = FunctionConvertOrZero; using FunctionToInt32OrZero = FunctionConvertOrZero; using FunctionToInt64OrZero = FunctionConvertOrZero; using FunctionToFloat32OrZero = FunctionConvertOrZero; using FunctionToFloat64OrZero = FunctionConvertOrZero; class FunctionCast final : public IFunction { public: FunctionCast(const Context & context) : context(context) {} private: using WrapperType = std::function; const Context & context; WrapperType wrapper_function; std::function monotonicity_for_range; template WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const) { using FunctionType = typename FunctionTo::Type; auto function = FunctionType::create(context); /// Check conversion using underlying function { DataTypePtr unused_data_type; std::vector unused_prerequisites; function->getReturnTypeAndPrerequisites({{ nullptr, from_type, "" }}, unused_data_type, unused_prerequisites); } return [function] (Block & block, const ColumnNumbers & arguments, const size_t result) { function->execute(block, arguments, result); }; } static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N) { if (!from_type->isStringOrFixedString()) throw Exception{ "CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED}; return [N] (Block & block, const ColumnNumbers & arguments, const size_t result) { FunctionToFixedString::execute(block, arguments, result, N); }; } WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) { /// Conversion from String through parsing. if (checkAndGetDataType(from_type_untyped.get())) { return [] (Block & block, const ColumnNumbers & arguments, const size_t result) { ConvertImplGenericFromString::execute(block, arguments, result); }; } DataTypePtr from_nested_type; DataTypePtr to_nested_type; auto from_type = checkAndGetDataType(from_type_untyped.get()); /// get the most nested type if (from_type && to_type) { from_nested_type = from_type->getNestedType(); to_nested_type = to_type->getNestedType(); from_type = checkAndGetDataType(from_nested_type.get()); to_type = checkAndGetDataType(to_nested_type.get()); } /// both from_type and to_type should be nullptr now is array types had same dimensions if (from_type || to_type) throw Exception{ "CAST AS Array can only be performed between same-dimensional array types or from String", ErrorCodes::TYPE_MISMATCH}; /// Prepare nested type conversion const auto nested_function = prepare(from_nested_type, to_nested_type.get()); return [nested_function, from_nested_type, to_nested_type]( Block & block, const ColumnNumbers & arguments, const size_t result) { const auto & array_arg = block.getByPosition(arguments.front()); if (const ColumnArray * col_array = checkAndGetColumn(array_arg.column.get())) { /// create block for converting nested column containing original and result columns Block nested_block { { col_array->getDataPtr(), from_nested_type, "" }, { nullptr, to_nested_type, "" } }; /// convert nested column nested_function(nested_block, {0}, 1); /// set converted nested column to result block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr()); } else throw Exception{ "Illegal column " + array_arg.column->getName() + " for function CAST AS Array", ErrorCodes::LOGICAL_ERROR}; }; } WrapperType createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type) { /// Conversion from String through parsing. if (checkAndGetDataType(from_type_untyped.get())) { return [] (Block & block, const ColumnNumbers & arguments, const size_t result) { ConvertImplGenericFromString::execute(block, arguments, result); }; } const auto from_type = checkAndGetDataType(from_type_untyped.get()); if (!from_type) throw Exception{ "CAST AS Tuple can only be performed between tuple types or from String.\nLeft type: " + from_type_untyped->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH}; if (from_type->getElements().size() != to_type->getElements().size()) throw Exception{ "CAST AS Tuple can only be performed between tuple types with the same number of elements or from String.\n" "Left type: " + from_type->getName() + ", right type: " + to_type->getName(), ErrorCodes::TYPE_MISMATCH}; const auto & from_element_types = from_type->getElements(); const auto & to_element_types = to_type->getElements(); std::vector element_wrappers; element_wrappers.reserve(from_element_types.size()); /// Create conversion wrapper for each element in tuple for (const auto & idx_type : ext::enumerate(from_type->getElements())) element_wrappers.push_back(prepare(idx_type.second, to_element_types[idx_type.first].get())); return [element_wrappers, from_element_types, to_element_types] (Block & block, const ColumnNumbers & arguments, const size_t result) { const auto col = block.getByPosition(arguments.front()).column.get(); /// copy tuple elements to a separate block Block element_block; size_t tuple_size = from_element_types.size(); const ColumnTuple & column_tuple = typeid_cast(*col); /// create columns for source elements for (size_t i = 0; i < tuple_size; ++i) element_block.insert({ column_tuple.getColumns()[i], from_element_types[i], "" }); /// create columns for converted elements for (const auto & to_element_type : to_element_types) element_block.insert({ nullptr, to_element_type, "" }); /// insert column for converted tuple element_block.insert({ nullptr, std::make_shared(to_element_types), "" }); /// invoke conversion for each element for (const auto & idx_element_wrapper : ext::enumerate(element_wrappers)) idx_element_wrapper.second(element_block, { idx_element_wrapper.first }, tuple_size + idx_element_wrapper.first); Columns converted_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) converted_columns[i] = element_block.getByPosition(tuple_size + i).column; block.getByPosition(result).column = ColumnTuple::create(converted_columns); }; } template WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum * to_type) { using EnumType = DataTypeEnum; using Function = typename FunctionTo::Type; if (const auto from_enum8 = checkAndGetDataType(from_type.get())) checkEnumToEnumConversion(from_enum8, to_type); else if (const auto from_enum16 = checkAndGetDataType(from_type.get())) checkEnumToEnumConversion(from_enum16, to_type); if (checkAndGetDataType(from_type.get())) return createStringToEnumWrapper(); else if (checkAndGetDataType(from_type.get())) return createStringToEnumWrapper(); else if (from_type->isNumber() || from_type->isEnum()) { auto function = Function::create(context); /// Check conversion using underlying function { DataTypePtr unused_data_type; std::vector unused_prerequisites; function->getReturnTypeAndPrerequisites({{ nullptr, from_type, "" }}, unused_data_type, unused_prerequisites); } return [function] (Block & block, const ColumnNumbers & arguments, const size_t result) { function->execute(block, arguments, result); }; } else throw Exception{ "Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE}; } template void checkEnumToEnumConversion(const EnumTypeFrom * from_type, const EnumTypeTo * to_type) { const auto & from_values = from_type->getValues(); const auto & to_values = to_type->getValues(); using ValueType = std::common_type_t; using NameValuePair = std::pair; using EnumValues = std::vector; EnumValues name_intersection; std::set_intersection(std::begin(from_values), std::end(from_values), std::begin(to_values), std::end(to_values), std::back_inserter(name_intersection), [] (auto && from, auto && to) { return from.first < to.first; }); for (const auto & name_value : name_intersection) { const auto & old_value = name_value.second; const auto & new_value = to_type->getValue(name_value.first); if (old_value != new_value) throw Exception{ "Enum conversion changes value for element '" + name_value.first + "' from " + toString(old_value) + " to " + toString(new_value), ErrorCodes::CANNOT_CONVERT_TYPE}; } }; template WrapperType createStringToEnumWrapper() { return [] (Block & block, const ColumnNumbers & arguments, const size_t result) { const auto first_col = block.getByPosition(arguments.front()).column.get(); auto & col_with_type_and_name = block.getByPosition(result); const auto & result_type = typeid_cast(*col_with_type_and_name.type); if (const auto col = typeid_cast(first_col)) { const auto size = col->size(); auto res = result_type.createColumn(); auto & out_data = static_cast(*res).getData(); out_data.resize(size); for (const auto i : ext::range(0, size)) out_data[i] = result_type.getValue(col->getDataAt(i)); col_with_type_and_name.column = std::move(res); } else throw Exception{ "Unexpected column " + first_col->getName() + " as first argument of function " + name, ErrorCodes::LOGICAL_ERROR}; }; } WrapperType createIdentityWrapper(const DataTypePtr &) { return [] (Block & block, const ColumnNumbers & arguments, const size_t result) { block.getByPosition(result).column = block.getByPosition(arguments.front()).column; }; } WrapperType createNothingWrapper(const IDataType * to_type) { ColumnPtr res = to_type->createColumnConstWithDefaultValue(1); return [res] (Block & block, const ColumnNumbers &, const size_t result) { /// Column of Nothing type is trivially convertible to any other column block.getByPosition(result).column = res->cloneResized(block.rows())->convertToFullColumnIfConst(); }; } /// Actions to be taken when performing a conversion. struct NullableConversion { bool source_is_nullable = false; bool result_is_nullable = false; }; WrapperType prepare(const DataTypePtr & from_type, const IDataType * to_type) { /// Determine whether pre-processing and/or post-processing must take place during conversion. NullableConversion nullable_conversion; nullable_conversion.source_is_nullable = from_type->isNullable(); nullable_conversion.result_is_nullable = to_type->isNullable(); /// Check that the requested conversion is allowed. if (nullable_conversion.source_is_nullable && !nullable_conversion.result_is_nullable) throw Exception{"Cannot convert data from a nullable type to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE}; if (from_type->onlyNull()) { return [](Block & block, const ColumnNumbers &, const size_t result) { auto & res = block.getByPosition(result); res.column = res.type->createColumnConstWithDefaultValue(block.rows())->convertToFullColumnIfConst(); }; } DataTypePtr from_inner_type; const IDataType * to_inner_type; /// Create the requested conversion. if (nullable_conversion.result_is_nullable) { if (nullable_conversion.source_is_nullable) { const auto & nullable_type = static_cast(*from_type); from_inner_type = nullable_type.getNestedType(); } else from_inner_type = from_type; const auto & nullable_type = static_cast(*to_type); to_inner_type = nullable_type.getNestedType().get(); } else { from_inner_type = from_type; to_inner_type = to_type; } auto wrapper = prepareImpl(from_inner_type, to_inner_type); if (nullable_conversion.result_is_nullable) { return [wrapper, nullable_conversion] (Block & block, const ColumnNumbers & arguments, const size_t result) { /// Create a temporary block on which to perform the operation. auto & res = block.getByPosition(result); const auto & ret_type = res.type; const auto & nullable_type = static_cast(*ret_type); const auto & nested_type = nullable_type.getNestedType(); Block tmp_block; if (nullable_conversion.source_is_nullable) tmp_block = createBlockWithNestedColumns(block, arguments); else tmp_block = block; size_t tmp_res_index = block.columns(); tmp_block.insert({nullptr, nested_type, ""}); /// Perform the requested conversion. wrapper(tmp_block, arguments, tmp_res_index); /// Wrap the result into a nullable column. ColumnPtr null_map; if (nullable_conversion.source_is_nullable) { /// This is a conversion from a nullable to a nullable type. /// So we just keep the null map of the input argument. const auto & col = block.getByPosition(arguments[0]).column; const auto & nullable_col = static_cast(*col); null_map = nullable_col.getNullMapColumnPtr(); } else { /// This is a conversion from an ordinary type to a nullable type. /// So we create a trivial null map. null_map = ColumnUInt8::create(block.rows(), 0); } const auto & tmp_res = tmp_block.getByPosition(tmp_res_index); res.column = ColumnNullable::create(tmp_res.column, null_map); }; } else return wrapper; } WrapperType prepareImpl(const DataTypePtr & from_type, const IDataType * to_type) { if (from_type->equals(*to_type)) return createIdentityWrapper(from_type); else if (checkDataType(from_type.get())) return createNothingWrapper(to_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto to_actual_type = checkAndGetDataType(to_type)) return createWrapper(from_type, to_actual_type); else if (const auto type_fixed_string = checkAndGetDataType(to_type)) return createFixedStringWrapper(from_type, type_fixed_string->getN()); else if (const auto type_array = checkAndGetDataType(to_type)) return createArrayWrapper(from_type, type_array); else if (const auto type_tuple = checkAndGetDataType(to_type)) return createTupleWrapper(from_type, type_tuple); else if (const auto type_enum = checkAndGetDataType(to_type)) return createEnumWrapper(from_type, type_enum); else if (const auto type_enum = checkAndGetDataType(to_type)) return createEnumWrapper(from_type, type_enum); /// It's possible to use ConvertImplGenericFromString to convert from String to AggregateFunction, /// but it is disabled because deserializing aggregate functions state might be unsafe. throw Exception{ "Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE}; } template static auto monotonicityForType(const DataType * const) { return FunctionTo::Type::Monotonic::get; } void prepareMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type) { if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (from_type->isEnum()) { if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); else if (const auto type = checkAndGetDataType(to_type)) monotonicity_for_range = monotonicityForType(type); } /// other types like Null, FixedString, Array and Tuple have no monotonicity defined } public: static constexpr auto name = "CAST"; static FunctionPtr create(const Context & context) { return std::make_shared(context); } String getName() const override { return name; } bool useDefaultImplementationForNulls() const override { return false; } size_t getNumberOfArguments() const override { return 2; } void getReturnTypeAndPrerequisitesImpl( const ColumnsWithTypeAndName & arguments, DataTypePtr & out_return_type, std::vector & /*out_prerequisites*/) override { const auto type_col = checkAndGetColumnConst(arguments.back().column.get()); if (!type_col) throw Exception("Second argument to " + getName() + " must be a constant string describing type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); out_return_type = DataTypeFactory::instance().get(type_col->getValue()); const DataTypePtr & from_type = arguments.front().type; wrapper_function = prepare(from_type, out_return_type.get()); prepareMonotonicityInformation(from_type, out_return_type.get()); } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { /// drop second argument, pass others ColumnNumbers new_arguments{arguments.front()}; if (arguments.size() > 2) new_arguments.insert(std::end(new_arguments), std::next(std::begin(arguments), 2), std::end(arguments)); wrapper_function(block, new_arguments, result); } bool hasInformationAboutMonotonicity() const override { return static_cast(monotonicity_for_range); } Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { return monotonicity_for_range(type, left, right); } }; }