FunctionsConversion.h 108.5 KB
Newer Older
A
Alexey Milovidov 已提交
1 2
#pragma once

3 4 5
#include <ext/enumerate.h>
#include <ext/collection_cast.h>
#include <ext/range.h>
6 7
#include <type_traits>

8 9
#include <IO/WriteBufferFromVector.h>
#include <IO/ReadBufferFromMemory.h>
10
#include <IO/Operators.h>
11
#include <IO/parseDateTimeBestEffort.h>
12 13
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypesNumber.h>
C
chertus 已提交
14
#include <DataTypes/DataTypesDecimal.h>
15 16 17 18
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
19
#include <DataTypes/DataTypeDateTime64.h>
20 21 22 23
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNullable.h>
24
#include <DataTypes/DataTypeNothing.h>
25
#include <DataTypes/DataTypeUUID.h>
26
#include <DataTypes/DataTypeInterval.h>
27
#include <DataTypes/DataTypeAggregateFunction.h>
28
#include <Formats/FormatSettings.h>
29 30 31 32 33 34
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
35
#include <Columns/ColumnsCommon.h>
36
#include <Common/FieldVisitors.h>
37
#include <Common/assert_cast.h>
38
#include <Interpreters/ExpressionActions.h>
39
#include <Functions/IFunctionAdaptors.h>
40
#include <Functions/FunctionsMiscellaneous.h>
41
#include <Functions/FunctionHelpers.h>
42
#include <Functions/DateTimeTransforms.h>
43 44
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnLowCardinality.h>
A
Alexey Milovidov 已提交
45 46 47 48 49


namespace DB
{

50 51
namespace ErrorCodes
{
52 53 54 55 56 57 58 59 60
    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;
61
    extern const int CANNOT_PARSE_UUID;
62
    extern const int TOO_LARGE_STRING_SIZE;
A
Alexey Milovidov 已提交
63
    extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
64 65 66 67 68 69 70
    extern const int LOGICAL_ERROR;
    extern const int TYPE_MISMATCH;
    extern const int CANNOT_CONVERT_TYPE;
    extern const int ILLEGAL_COLUMN;
    extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
    extern const int ILLEGAL_TYPE_OF_ARGUMENT;
    extern const int NOT_IMPLEMENTED;
71
    extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
72 73
}

74

75 76
/** Type conversion functions.
  * toType - conversion in "natural way";
A
Alexey Milovidov 已提交
77 78
  */

C
chertus 已提交
79 80 81 82 83 84 85 86 87 88 89 90 91 92 93
inline UInt32 extractToDecimalScale(const ColumnWithTypeAndName & named_column)
{
    const auto * arg_type = named_column.type.get();
    bool ok = checkAndGetDataType<DataTypeUInt64>(arg_type)
        || checkAndGetDataType<DataTypeUInt32>(arg_type)
        || checkAndGetDataType<DataTypeUInt16>(arg_type)
        || checkAndGetDataType<DataTypeUInt8>(arg_type);
    if (!ok)
        throw Exception("Illegal type of toDecimal() scale " + named_column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);

    Field field;
    named_column.column->get(0, field);
    return field.get<UInt32>();
}

A
Alexey Milovidov 已提交
94

95 96
/** 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)
A
Alexey Milovidov 已提交
97
  */
A
Alexey Milovidov 已提交
98 99
template <typename FromDataType, typename ToDataType, typename Name>
struct ConvertImpl
A
Alexey Milovidov 已提交
100
{
101 102 103
    using FromFieldType = typename FromDataType::FieldType;
    using ToFieldType = typename ToDataType::FieldType;

C
chertus 已提交
104
    template <typename Additions = void *>
A
Alexey Milovidov 已提交
105
    static void NO_SANITIZE_UNDEFINED execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/,
C
chertus 已提交
106
                        Additions additions [[maybe_unused]] = Additions())
107
    {
C
chertus 已提交
108 109
        const ColumnWithTypeAndName & named_from = block.getByPosition(arguments[0]);

110 111
        using ColVecFrom = typename FromDataType::ColumnType;
        using ColVecTo = typename ToDataType::ColumnType;
112

113 114
        if constexpr ((IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
            && !(std::is_same_v<DataTypeDateTime64, FromDataType> || std::is_same_v<DataTypeDateTime64, ToDataType>))
115 116
        {
            if constexpr (!IsDataTypeDecimalOrNumber<FromDataType> || !IsDataTypeDecimalOrNumber<ToDataType>)
117
            {
118 119
                throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
                    ErrorCodes::ILLEGAL_COLUMN);
120
            }
121 122
        }

123
        if (const ColVecFrom * col_from = checkAndGetColumn<ColVecFrom>(named_from.column.get()))
124
        {
125
            typename ColVecTo::MutablePtr col_to = nullptr;
126
            if constexpr (IsDataTypeDecimal<ToDataType>)
C
chertus 已提交
127
            {
C
chertus 已提交
128
                UInt32 scale = additions;
129
                col_to = ColVecTo::create(0, scale);
C
chertus 已提交
130
            }
131 132
            else
                col_to = ColVecTo::create();
133

134 135
            const auto & vec_from = col_from->getData();
            auto & vec_to = col_to->getData();
136 137 138 139
            size_t size = vec_from.size();
            vec_to.resize(size);

            for (size_t i = 0; i < size; ++i)
C
chertus 已提交
140
            {
141 142 143 144 145 146 147 148 149
                if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
                {
                    if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
                        vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], vec_from.getScale(), vec_to.getScale());
                    else if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeNumber<ToDataType>)
                        vec_to[i] = convertFromDecimal<FromDataType, ToDataType>(vec_from[i], vec_from.getScale());
                    else if constexpr (IsDataTypeNumber<FromDataType> && IsDataTypeDecimal<ToDataType>)
                        vec_to[i] = convertToDecimal<FromDataType, ToDataType>(vec_from[i], vec_to.getScale());
                }
C
chertus 已提交
150 151 152
                else
                    vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
            }
153 154

            block.getByPosition(result).column = std::move(col_to);
155 156
        }
        else
C
chertus 已提交
157
            throw Exception("Illegal column " + named_from.column->getName() + " of first argument of function " + Name::name,
158 159
                ErrorCodes::ILLEGAL_COLUMN);
    }
A
Alexey Milovidov 已提交
160 161
};

162 163 164 165 166
/** Conversion of DateTime to Date: throw off time component.
  */
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDate, Name>
    : DateTimeTransformImpl<DataTypeDateTime, DataTypeDate, ToDateImpl> {};

A
Alexey Milovidov 已提交
167

168
/** Conversion of Date to DateTime: adding 00:00:00 time component.
A
Alexey Milovidov 已提交
169
  */
170
struct ToDateTimeImpl
A
Alexey Milovidov 已提交
171
{
172 173
    static constexpr auto name = "toDateTime";

174 175
    static inline UInt32 execute(UInt16 d, const DateLUTImpl & time_zone)
    {
176
        return time_zone.fromDayNum(DayNum(d));
177
    }
178

179 180
    // no-op conversion from DateTime to DateTime, used in DateTime64 to DateTime conversion.
    static inline UInt32 execute(UInt32 d, const DateLUTImpl & /*time_zone*/)
181
    {
182
        return d;
183
    }
A
Alexey Milovidov 已提交
184 185
};

186
template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime, Name>
V
Vasily Nemkov 已提交
187
    : DateTimeTransformImpl<DataTypeDate, DataTypeDateTime, ToDateTimeImpl> {};
188

189
/// Implementation of toDate function.
190

A
Merge  
Alexey Arno 已提交
191 192 193
template <typename FromType, typename ToType>
struct ToDateTransform32Or64
{
194 195
    static constexpr auto name = "toDate";

A
Alexey Milovidov 已提交
196
    static inline NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone)
197 198 199
    {
        return (from < 0xFFFF) ? from : time_zone.toDayNum(from);
    }
A
Merge  
Alexey Arno 已提交
200 201
};

202
/** Special case of converting (U)Int32 or (U)Int64 (and also, for convenience, Float32, Float64) to Date.
203 204 205 206 207
  * 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).
A
Merge  
Alexey Arno 已提交
208
  */
209
template <typename Name> struct ConvertImpl<DataTypeUInt32, DataTypeDate, Name>
V
Vasily Nemkov 已提交
210
    : DateTimeTransformImpl<DataTypeUInt32, DataTypeDate, ToDateTransform32Or64<UInt32, UInt16>> {};
211
template <typename Name> struct ConvertImpl<DataTypeUInt64, DataTypeDate, Name>
V
Vasily Nemkov 已提交
212
    : DateTimeTransformImpl<DataTypeUInt64, DataTypeDate, ToDateTransform32Or64<UInt64, UInt16>> {};
213
template <typename Name> struct ConvertImpl<DataTypeInt32, DataTypeDate, Name>
V
Vasily Nemkov 已提交
214
    : DateTimeTransformImpl<DataTypeInt32, DataTypeDate, ToDateTransform32Or64<Int32, UInt16>> {};
215
template <typename Name> struct ConvertImpl<DataTypeInt64, DataTypeDate, Name>
V
Vasily Nemkov 已提交
216
    : DateTimeTransformImpl<DataTypeInt64, DataTypeDate, ToDateTransform32Or64<Int64, UInt16>> {};
217
template <typename Name> struct ConvertImpl<DataTypeFloat32, DataTypeDate, Name>
V
Vasily Nemkov 已提交
218
    : DateTimeTransformImpl<DataTypeFloat32, DataTypeDate, ToDateTransform32Or64<Float32, UInt16>> {};
219
template <typename Name> struct ConvertImpl<DataTypeFloat64, DataTypeDate, Name>
V
Vasily Nemkov 已提交
220
    : DateTimeTransformImpl<DataTypeFloat64, DataTypeDate, ToDateTransform32Or64<Float64, UInt16>> {};
221

222

223 224 225 226 227 228 229 230 231
/** Conversion of Date or DateTime to DateTime64: add zero sub-second part.
  */
struct ToDateTime64Transform
{
    static constexpr auto name = "toDateTime64";

    const DateTime64::NativeType scale_multiplier = 1;

    ToDateTime64Transform(UInt32 scale = 0)
232
        : scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale))
233 234 235 236 237 238 239 240 241 242
    {}

    inline DateTime64::NativeType execute(UInt16 d, const DateLUTImpl & time_zone) const
    {
        const auto dt = ToDateTimeImpl::execute(d, time_zone);
        return execute(dt, time_zone);
    }

    inline DateTime64::NativeType execute(UInt32 dt, const DateLUTImpl & /*time_zone*/) const
    {
243
        return DecimalUtils::decimalFromComponentsWithMultiplier<DateTime64>(dt, 0, scale_multiplier);
244 245 246 247 248 249 250 251
    }
};

template <typename Name> struct ConvertImpl<DataTypeDate, DataTypeDateTime64, Name>
    : DateTimeTransformImpl<DataTypeDate, DataTypeDateTime64, ToDateTime64Transform> {};
template <typename Name> struct ConvertImpl<DataTypeDateTime, DataTypeDateTime64, Name>
    : DateTimeTransformImpl<DataTypeDateTime, DataTypeDateTime64, ToDateTime64Transform> {};

252 253 254 255 256
/** Conversion of DateTime64 to Date or DateTime: discards fractional part.
 */
template <typename Transform>
struct FromDateTime64Transform
{
V
Vasily Nemkov 已提交
257
    static constexpr auto name = Transform::name;
258 259 260 261

    const DateTime64::NativeType scale_multiplier = 1;

    FromDateTime64Transform(UInt32 scale)
262
        : scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale))
263 264 265 266
    {}

    inline auto execute(DateTime64::NativeType dt, const DateLUTImpl & time_zone) const
    {
267
        const auto c = DecimalUtils::splitWithScaleMultiplier(DateTime64(dt), scale_multiplier);
268 269 270 271 272 273 274 275
        return Transform::execute(static_cast<UInt32>(c.whole), time_zone);
    }
};

template <typename Name> struct ConvertImpl<DataTypeDateTime64, DataTypeDate, Name>
    : DateTimeTransformImpl<DataTypeDateTime64, DataTypeDate, FromDateTime64Transform<ToDateImpl>> {};
template <typename Name> struct ConvertImpl<DataTypeDateTime64, DataTypeDateTime, Name>
    : DateTimeTransformImpl<DataTypeDateTime64, DataTypeDateTime, FromDateTime64Transform<ToDateTimeImpl>> {};
276

277 278

/** Transformation of numbers, dates, datetimes to strings: through formatting.
A
Alexey Milovidov 已提交
279
  */
280 281
template <typename DataType>
struct FormatImpl
282
{
A
Alexey Milovidov 已提交
283
    static void execute(const typename DataType::FieldType x, WriteBuffer & wb, const DataType *, const DateLUTImpl *)
284 285 286
    {
        writeText(x, wb);
    }
287 288
};

289 290
template <>
struct FormatImpl<DataTypeDate>
291
{
A
Alexey Milovidov 已提交
292
    static void execute(const DataTypeDate::FieldType x, WriteBuffer & wb, const DataTypeDate *, const DateLUTImpl *)
293
    {
294
        writeDateText(DayNum(x), wb);
295
    }
296 297
};

298 299
template <>
struct FormatImpl<DataTypeDateTime>
300
{
A
Alexey Milovidov 已提交
301
    static void execute(const DataTypeDateTime::FieldType x, WriteBuffer & wb, const DataTypeDateTime *, const DateLUTImpl * time_zone)
302 303 304
    {
        writeDateTimeText(x, wb, *time_zone);
    }
305 306
};

307 308 309
template <>
struct FormatImpl<DataTypeDateTime64>
{
310
    static void execute(const DataTypeDateTime64::FieldType x, WriteBuffer & wb, const DataTypeDateTime64 * type, const DateLUTImpl * time_zone)
311
    {
312
        writeDateTimeText(DateTime64(x), type->getScale(), wb, *time_zone);
313 314 315 316
    }
};


317 318
template <typename FieldType>
struct FormatImpl<DataTypeEnum<FieldType>>
319
{
A
Alexey Milovidov 已提交
320
    static void execute(const FieldType x, WriteBuffer & wb, const DataTypeEnum<FieldType> * type, const DateLUTImpl *)
321 322 323
    {
        writeString(type->getNameForValue(x), wb);
    }
324
};
A
Alexey Milovidov 已提交
325

C
chertus 已提交
326 327 328 329 330
template <typename FieldType>
struct FormatImpl<DataTypeDecimal<FieldType>>
{
    static void execute(const FieldType x, WriteBuffer & wb, const DataTypeDecimal<FieldType> * type, const DateLUTImpl *)
    {
C
chertus 已提交
331
        writeText(x, type->getScale(), wb);
C
chertus 已提交
332 333 334
    }
};

335 336 337

/// DataTypeEnum<T> to DataType<T> free conversion
template <typename FieldType, typename Name>
338
struct ConvertImpl<DataTypeEnum<FieldType>, DataTypeNumber<FieldType>, Name>
339
{
T
Tsarkova Anastasia 已提交
340
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
341
    {
342
        block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
343
    }
344 345 346
};


A
Alexey Milovidov 已提交
347
template <typename FromDataType, typename Name>
A
Alexey Milovidov 已提交
348
struct ConvertImpl<FromDataType, std::enable_if_t<!std::is_same_v<FromDataType, DataTypeString>, DataTypeString>, Name>
A
Alexey Milovidov 已提交
349
{
350
    using FromFieldType = typename FromDataType::FieldType;
351
    using ColVecType = std::conditional_t<IsDecimalNumber<FromFieldType>, ColumnDecimal<FromFieldType>, ColumnVector<FromFieldType>>;
352

T
Tsarkova Anastasia 已提交
353
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
354
    {
355
        const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
356 357 358 359 360
        const auto & type = static_cast<const FromDataType &>(*col_with_type_and_name.type);

        const DateLUTImpl * time_zone = nullptr;

        /// For argument of DateTime type, second argument with time zone could be specified.
361
        if constexpr (std::is_same_v<FromDataType, DataTypeDateTime> || std::is_same_v<FromDataType, DataTypeDateTime64>)
362
            time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
363

364
        if (const auto col_from = checkAndGetColumn<ColVecType>(col_with_type_and_name.column.get()))
365
        {
366
            auto col_to = ColumnString::create();
367

368
            const typename ColVecType::Container & vec_from = col_from->getData();
369
            ColumnString::Chars & data_to = col_to->getChars();
370
            ColumnString::Offsets & offsets_to = col_to->getOffsets();
371
            size_t size = vec_from.size();
372

A
Alexey Milovidov 已提交
373
            if constexpr (std::is_same_v<FromDataType, DataTypeDate>)
374
                data_to.resize(size * (strlen("YYYY-MM-DD") + 1));
A
Alexey Milovidov 已提交
375
            else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime>)
376
                data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss") + 1));
377
            else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>)
378
                data_to.resize(size * (strlen("YYYY-MM-DD hh:mm:ss.") + vec_from.getScale() + 1));
379 380 381
            else
                data_to.resize(size * 3);   /// Arbitary

382 383
            offsets_to.resize(size);

384
            WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
385 386 387 388 389 390 391 392

            for (size_t i = 0; i < size; ++i)
            {
                FormatImpl<FromDataType>::execute(vec_from[i], write_buffer, &type, time_zone);
                writeChar(0, write_buffer);
                offsets_to[i] = write_buffer.count();
            }

A
Alexander Burmak 已提交
393
            write_buffer.finalize();
394
            block.getByPosition(result).column = std::move(col_to);
395 396
        }
        else
397
            throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
398 399 400
                    + " of first argument of function " + Name::name,
                ErrorCodes::ILLEGAL_COLUMN);
    }
A
Alexey Milovidov 已提交
401 402
};

403 404 405 406

/// Generic conversion of any type to String.
struct ConvertImplGenericToString
{
407 408
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
    {
409
        const auto & col_with_type_and_name = block.getByPosition(arguments[0]);
410 411 412 413 414
        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();

415
        auto col_to = ColumnString::create();
416

417
        ColumnString::Chars & data_to = col_to->getChars();
418
        ColumnString::Offsets & offsets_to = col_to->getOffsets();
419

A
Alexey Milovidov 已提交
420
        data_to.resize(size * 2); /// Using coefficient 2 for initial size is arbitrary.
421
        offsets_to.resize(size);
422

423
        WriteBufferFromVector<ColumnString::Chars> write_buffer(data_to);
424

425
        FormatSettings format_settings;
426
        for (size_t i = 0; i < size; ++i)
427
        {
428
            type.serializeAsText(col_from, i, write_buffer, format_settings);
429 430
            writeChar(0, write_buffer);
            offsets_to[i] = write_buffer.count();
431
        }
432

A
Alexander Burmak 已提交
433
        write_buffer.finalize();
434
        block.getByPosition(result).column = std::move(col_to);
435
    }
436 437 438
};


439
/** Conversion of strings to numbers, dates, datetimes: through parsing.
440
  */
441 442
template <typename DataType>
void parseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
443
{
444
    readText(x, rb);
445 446
}

447 448
template <>
inline void parseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
A
Alexey Milovidov 已提交
449
{
450
    DayNum tmp(0);
451 452
    readDateText(tmp, rb);
    x = tmp;
A
Alexey Milovidov 已提交
453 454
}

455 456
template <>
inline void parseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
A
Alexey Milovidov 已提交
457
{
458 459 460
    time_t tmp = 0;
    readDateTimeText(tmp, rb, *time_zone);
    x = tmp;
A
Alexey Milovidov 已提交
461 462
}

A
Alexey Milovidov 已提交
463 464
template <>
inline void parseImpl<DataTypeUUID>(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
G
Guillaume Tassery 已提交
465 466 467 468 469
{
    UUID tmp;
    readText(tmp, rb);
    x = tmp;
}
470 471


472
template <typename DataType>
473
bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
474
{
A
Alexey Milovidov 已提交
475
    if constexpr (std::is_floating_point_v<typename DataType::FieldType>)
476
        return tryReadFloatText(x, rb);
A
Amos Bird 已提交
477
    else /*if constexpr (is_integral_v<typename DataType::FieldType>)*/
A
Alexey Milovidov 已提交
478
        return tryReadIntText(x, rb);
479 480 481 482 483 484
}

template <>
inline bool tryParseImpl<DataTypeDate>(DataTypeDate::FieldType & x, ReadBuffer & rb, const DateLUTImpl *)
{
    DayNum tmp(0);
485 486
    if (!tryReadDateText(tmp, rb))
        return false;
487 488 489 490 491 492 493 494
    x = tmp;
    return true;
}

template <>
inline bool tryParseImpl<DataTypeDateTime>(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone)
{
    time_t tmp = 0;
495 496
    if (!tryReadDateTimeText(tmp, rb, *time_zone))
        return false;
497 498
    x = tmp;
    return true;
499 500 501
}


502 503
/** Throw exception with verbose message when string value is not parsed completely.
  */
504 505 506 507 508 509 510 511 512 513 514 515 516 517 518 519 520 521 522 523
[[noreturn]] inline void throwExceptionForIncompletelyParsedValue(ReadBuffer & read_buffer, Block & block, size_t result)
{
    const IDataType & to_type = *block.getByPosition(result).type;

    WriteBufferFromOwnString message_buf;
    message_buf << "Cannot parse string " << quote << String(read_buffer.buffer().begin(), read_buffer.buffer().size())
                << " as " << to_type.getName()
                << ": syntax error";

    if (read_buffer.offset())
        message_buf << " at position " << read_buffer.offset()
                    << " (parsed just " << quote << String(read_buffer.buffer().begin(), read_buffer.offset()) << ")";
    else
        message_buf << " at begin of string";

    if (isNativeNumber(to_type))
        message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception.";

    throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT);
}
524 525


526 527 528 529 530 531 532
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.
};

533 534 535 536 537 538 539 540
enum class ConvertFromStringParsingMode
{
    Normal,
    BestEffort  /// Only applicable for DateTime. Will use sophisticated method, that is slower.
};

template <typename FromDataType, typename ToDataType, typename Name,
    ConvertFromStringExceptionMode exception_mode, ConvertFromStringParsingMode parsing_mode>
541
struct ConvertThroughParsing
A
Alexey Milovidov 已提交
542
{
543
    static_assert(std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>,
A
Alexey Milovidov 已提交
544
        "ConvertThroughParsing is only applicable for String or FixedString data types");
545

546 547
    static constexpr bool to_datetime64 = std::is_same_v<ToDataType, DataTypeDateTime64>;

548 549
    using ToFieldType = typename ToDataType::FieldType;

550 551 552 553 554 555 556 557 558 559 560
    static bool isAllRead(ReadBuffer & in)
    {
        /// In case of FixedString, skip zero bytes at end.
        if constexpr (std::is_same_v<FromDataType, DataTypeFixedString>)
            while (!in.eof() && *in.position() == 0)
                ++in.position();

        if (in.eof())
            return true;

        /// Special case, that allows to parse string with DateTime as Date.
A
Alexey Milovidov 已提交
561
        if (std::is_same_v<ToDataType, DataTypeDate> && (in.buffer().size()) == strlen("YYYY-MM-DD hh:mm:ss"))
562 563 564 565 566
            return true;

        return false;
    }

C
chertus 已提交
567 568 569
    template <typename Additions = void *>
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count,
                        Additions additions [[maybe_unused]] = Additions())
570
    {
571
        using ColVecTo = typename ToDataType::ColumnType;
572

573 574
        const DateLUTImpl * local_time_zone [[maybe_unused]] = nullptr;
        const DateLUTImpl * utc_time_zone [[maybe_unused]] = nullptr;
575 576

        /// For conversion to DateTime type, second argument with time zone could be specified.
577
        if constexpr (std::is_same_v<ToDataType, DataTypeDateTime> || to_datetime64)
578
        {
579
            const auto result_type = removeNullable(block.getByPosition(result).type);
V
Vasily Nemkov 已提交
580
            // Time zone is already figured out during result type resultion, no need to do it here.
581
            if (const auto dt_col = checkAndGetDataType<ToDataType>(result_type.get()))
V
Vasily Nemkov 已提交
582 583 584
                local_time_zone = &dt_col->getTimeZone();
            else
            {
585
                local_time_zone = &extractTimeZoneFromFunctionArguments(block, arguments, 1, 0);
V
Vasily Nemkov 已提交
586
            }
587 588 589 590

            if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
                utc_time_zone = &DateLUT::instance("UTC");
        }
591

592 593 594 595 596 597 598 599 600 601 602 603 604 605
        const IColumn * col_from = block.getByPosition(arguments[0]).column.get();
        const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(col_from);
        const ColumnFixedString * col_from_fixed_string = checkAndGetColumn<ColumnFixedString>(col_from);

        if (std::is_same_v<FromDataType, DataTypeString> && !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<FromDataType, DataTypeFixedString> && !col_from_fixed_string)
            throw Exception("Illegal column " + col_from->getName()
                + " of first argument of function " + Name::name,
                ErrorCodes::ILLEGAL_COLUMN);

T
Tsarkova Anastasia 已提交
606
        size_t size = input_rows_count;
607
        typename ColVecTo::MutablePtr col_to = nullptr;
608

609
        if constexpr (IsDataTypeDecimal<ToDataType>)
C
chertus 已提交
610
        {
C
chertus 已提交
611
            UInt32 scale = additions;
612
            if constexpr (to_datetime64)
613 614 615 616 617 618 619
            {
                ToDataType check_bounds_in_ctor(scale, local_time_zone ? local_time_zone->getTimeZone() : String{});
            }
            else
            {
                ToDataType check_bounds_in_ctor(ToDataType::maxPrecision(), scale);
            }
C
chertus 已提交
620
            col_to = ColVecTo::create(size, scale);
C
chertus 已提交
621
        }
622 623 624 625
        else
            col_to = ColVecTo::create(size);

        typename ColVecTo::Container & vec_to = col_to->getData();
626 627

        ColumnUInt8::MutablePtr col_null_map_to;
A
Alexey Milovidov 已提交
628
        ColumnUInt8::Container * vec_null_map_to [[maybe_unused]] = nullptr;
629
        if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
630
        {
631 632 633
            col_null_map_to = ColumnUInt8::create(size);
            vec_null_map_to = &col_null_map_to->getData();
        }
634

635
        const ColumnString::Chars * chars = nullptr;
636 637 638 639 640 641 642 643 644 645 646 647 648
        const IColumn::Offsets * offsets = nullptr;
        size_t fixed_string_size = 0;

        if constexpr (std::is_same_v<FromDataType, DataTypeString>)
        {
            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();
        }
649

650
        size_t current_offset = 0;
651

652 653 654 655
        for (size_t i = 0; i < size; ++i)
        {
            size_t next_offset = std::is_same_v<FromDataType, DataTypeString> ? (*offsets)[i] : (current_offset + fixed_string_size);
            size_t string_size = std::is_same_v<FromDataType, DataTypeString> ? next_offset - current_offset - 1 : fixed_string_size;
656

657
            ReadBufferFromMemory read_buffer(&(*chars)[current_offset], string_size);
658

659
            if constexpr (exception_mode == ConvertFromStringExceptionMode::Throw)
660
            {
661 662
                if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
                {
663
                    if constexpr (to_datetime64)
664
                    {
665
                        DateTime64 res = 0;
666 667 668 669 670 671 672 673 674
                        parseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
                        vec_to[i] = res;
                    }
                    else
                    {
                        time_t res;
                        parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
                        vec_to[i] = res;
                    }
675 676 677
                }
                else
                {
678
                    if constexpr (to_datetime64)
679
                    {
680
                        DateTime64 value = 0;
681 682
                        readDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
                        vec_to[i] = value;
683 684
                    }
                    else if constexpr (IsDataTypeDecimal<ToDataType>)
685 686 687
                        ToDataType::readText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
                    else
                        parseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);
688
                }
689

690
                if (!isAllRead(read_buffer))
A
Alexey Milovidov 已提交
691
                    throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
692 693 694
            }
            else
            {
695 696 697 698
                bool parsed;

                if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffort)
                {
699
                    if constexpr (to_datetime64)
700
                    {
701
                        DateTime64 res = 0;
702 703 704 705 706 707 708 709 710
                        parsed = tryParseDateTime64BestEffort(res, vec_to.getScale(), read_buffer, *local_time_zone, *utc_time_zone);
                        vec_to[i] = res;
                    }
                    else
                    {
                        time_t res;
                        parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone);
                        vec_to[i] = res;
                    }
711 712 713
                }
                else
                {
714
                    if constexpr (to_datetime64)
715
                    {
716
                        DateTime64 value = 0;
717 718 719 720
                        parsed = tryReadDateTime64Text(value, vec_to.getScale(), read_buffer, *local_time_zone);
                        vec_to[i] = value;
                    }
                    else if constexpr (IsDataTypeDecimal<ToDataType>)
721 722 723 724 725
                        parsed = ToDataType::tryReadText(vec_to[i], read_buffer, ToDataType::maxPrecision(), vec_to.getScale());
                    else
                        parsed = tryParseImpl<ToDataType>(vec_to[i], read_buffer, local_time_zone);

                    parsed = parsed && isAllRead(read_buffer);
726
                }
727 728

                if (!parsed)
729
                    vec_to[i] = 0;
730

731
                if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
732
                    (*vec_null_map_to)[i] = !parsed;
733
            }
734

735
            current_offset = next_offset;
736
        }
737

738
        if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
739
            block.getByPosition(result).column = ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
740
        else
741
            block.getByPosition(result).column = std::move(col_to);
742
    }
743 744 745
};


746 747
template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeString>, DataTypeString>, ToDataType, Name>
748
    : ConvertThroughParsing<DataTypeString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
749 750 751

template <typename ToDataType, typename Name>
struct ConvertImpl<std::enable_if_t<!std::is_same_v<ToDataType, DataTypeFixedString>, DataTypeFixedString>, ToDataType, Name>
752
    : ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal> {};
A
Alexey Milovidov 已提交
753

754 755 756
/// Generic conversion of any type from String. Used for complex types: Array and Tuple.
struct ConvertImplGenericFromString
{
757 758
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result)
    {
759
        const IColumn & col_from = *block.getByPosition(arguments[0]).column;
760
        size_t size = col_from.size();
761

762
        const IDataType & data_type_to = *block.getByPosition(result).type;
763

764
        if (const ColumnString * col_from_string = checkAndGetColumn<ColumnString>(&col_from))
765
        {
766
            auto res = data_type_to.createColumn();
767

768
            IColumn & column_to = *res;
769
            column_to.reserve(size);
770

771
            const ColumnString::Chars & chars = col_from_string->getChars();
772
            const IColumn::Offsets & offsets = col_from_string->getOffsets();
773

774
            size_t current_offset = 0;
775

776
            FormatSettings format_settings;
777 778 779
            for (size_t i = 0; i < size; ++i)
            {
                ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1);
780

781
                data_type_to.deserializeAsWholeText(column_to, read_buffer, format_settings);
782

783
                if (!read_buffer.eof())
A
Alexey Milovidov 已提交
784
                    throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
785

786 787
                current_offset = offsets[i];
            }
788 789

            block.getByPosition(result).column = std::move(res);
790 791
        }
        else
792
            throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
793 794 795
                    + " of first argument of conversion function from string",
                ErrorCodes::ILLEGAL_COLUMN);
    }
796 797 798
};


799
/// Function toUnixTimestamp has exactly the same implementation as toDateTime of String type.
800
struct NameToUnixTimestamp { static constexpr auto name = "toUnixTimestamp"; };
801

A
Alexey Milovidov 已提交
802
template <>
803
struct ConvertImpl<DataTypeString, DataTypeUInt32, NameToUnixTimestamp>
804
    : ConvertImpl<DataTypeString, DataTypeDateTime, NameToUnixTimestamp> {};
805

806

A
Alexey Milovidov 已提交
807
/** If types are identical, just take reference to column.
A
Alexey Milovidov 已提交
808
  */
809
template <typename T, typename Name>
A
Alexey Milovidov 已提交
810
struct ConvertImpl<std::enable_if_t<!T::is_parametric, T>, T, Name>
A
Alexey Milovidov 已提交
811
{
T
Tsarkova Anastasia 已提交
812
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
813
    {
814
        block.getByPosition(result).column = block.getByPosition(arguments[0]).column;
815
    }
A
Alexey Milovidov 已提交
816 817 818
};


A
Alexey Milovidov 已提交
819 820
/** Conversion from FixedString to String.
  * Cutting sequences of zero bytes from end of strings.
821 822 823 824
  */
template <typename Name>
struct ConvertImpl<DataTypeFixedString, DataTypeString, Name>
{
T
Tsarkova Anastasia 已提交
825
    static void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
826
    {
827
        if (const ColumnFixedString * col_from = checkAndGetColumn<ColumnFixedString>(block.getByPosition(arguments[0]).column.get()))
828
        {
829
            auto col_to = ColumnString::create();
830

831 832
            const ColumnFixedString::Chars & data_from = col_from->getChars();
            ColumnString::Chars & data_to = col_to->getChars();
833
            ColumnString::Offsets & offsets_to = col_to->getOffsets();
834 835
            size_t size = col_from->size();
            size_t n = col_from->getN();
836
            data_to.resize(size * (n + 1)); /// + 1 - zero terminator
837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855
            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);
856
            block.getByPosition(result).column = std::move(col_to);
857 858
        }
        else
859
            throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
860 861 862
                    + " of first argument of function " + Name::name,
                ErrorCodes::ILLEGAL_COLUMN);
    }
863 864
};

A
Alexey Milovidov 已提交
865 866

/// Declared early because used below.
867
struct NameToDate { static constexpr auto name = "toDate"; };
868
struct NameToDateTime { static constexpr auto name = "toDateTime"; };
869
struct NameToDateTime64 { static constexpr auto name = "toDateTime64"; };
870
struct NameToString { static constexpr auto name = "toString"; };
871 872 873
struct NameToDecimal32 { static constexpr auto name = "toDecimal32"; };
struct NameToDecimal64 { static constexpr auto name = "toDecimal64"; };
struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; };
874 875 876 877 878 879


#define DEFINE_NAME_TO_INTERVAL(INTERVAL_KIND) \
    struct NameToInterval ## INTERVAL_KIND \
    { \
        static constexpr auto name = "toInterval" #INTERVAL_KIND; \
880
        static constexpr auto kind = IntervalKind::INTERVAL_KIND; \
881 882 883 884 885 886 887 888
    };

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)
889
DEFINE_NAME_TO_INTERVAL(Quarter)
890 891 892
DEFINE_NAME_TO_INTERVAL(Year)

#undef DEFINE_NAME_TO_INTERVAL
A
Alexey Milovidov 已提交
893

894

895
template <typename ToDataType, typename Name, typename MonotonicityImpl>
A
Alexey Milovidov 已提交
896 897
class FunctionConvert : public IFunction
{
A
Alexey Milovidov 已提交
898
public:
899 900 901
    using Monotonic = MonotonicityImpl;

    static constexpr auto name = Name::name;
C
chertus 已提交
902
    static constexpr bool to_decimal =
903
        std::is_same_v<Name, NameToDecimal32> || std::is_same_v<Name, NameToDecimal64> || std::is_same_v<Name, NameToDecimal128>;
C
chertus 已提交
904

905 906
    static constexpr bool to_datetime64 = std::is_same_v<ToDataType, DataTypeDateTime64>;

A
Alexey Milovidov 已提交
907
    static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvert>(); }
908
    static FunctionPtr create() { return std::make_shared<FunctionConvert>(); }
909 910 911 912 913 914 915 916

    String getName() const override
    {
        return name;
    }

    bool isVariadic() const override { return true; }
    size_t getNumberOfArguments() const override { return 0; }
A
Alexey Milovidov 已提交
917
    bool isInjective(const Block &) override { return std::is_same_v<Name, NameToString>; }
918

N
Nikolai Kochetov 已提交
919
    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
920
    {
V
Vasily Nemkov 已提交
921
        FunctionArgumentDescriptors mandatory_args = {{"Value", nullptr, nullptr, nullptr}};
V
Vasily Nemkov 已提交
922
        FunctionArgumentDescriptors optional_args;
923

924
        if constexpr (to_decimal || to_datetime64)
C
chertus 已提交
925
        {
V
Vasily Nemkov 已提交
926
            mandatory_args.push_back({"scale", &isNativeInteger, &isColumnConst, "const Integer"});
927
        }
V
Vasily Nemkov 已提交
928 929
        // toString(DateTime or DateTime64, [timezone: String])
        if ((std::is_same_v<Name, NameToString> && arguments.size() > 0 && (isDateTime64(arguments[0].type) || isDateTime(arguments[0].type)))
V
Vasily Nemkov 已提交
930 931 932 933 934 935 936 937
            // toUnixTimestamp(value[, timezone : String])
            || std::is_same_v<Name, NameToUnixTimestamp>
            // toDate(value[, timezone : String])
            || std::is_same_v<ToDataType, DataTypeDate> // TODO: shall we allow timestamp argument for toDate? DateTime knows nothing about timezones and this arument is ignored below.
            // toDateTime(value[, timezone: String])
            || std::is_same_v<ToDataType, DataTypeDateTime>
            // toDateTime64(value, scale : Integer[, timezone: String])
            || std::is_same_v<ToDataType, DataTypeDateTime64>)
C
chertus 已提交
938
        {
V
Vasily Nemkov 已提交
939
            optional_args.push_back({"timezone", &isString, &isColumnConst, "const String"});
C
chertus 已提交
940
        }
941 942

        validateFunctionArgumentTypes(*this, arguments, mandatory_args, optional_args);
943

A
Alexey Milovidov 已提交
944
        if constexpr (std::is_same_v<ToDataType, DataTypeInterval>)
945
        {
946
            return std::make_shared<DataTypeInterval>(Name::kind);
A
Alexey Milovidov 已提交
947
        }
C
chertus 已提交
948 949
        else if constexpr (to_decimal)
        {
V
Vasily Nemkov 已提交
950 951
//            if (!arguments[1].column)
//                throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
952

C
chertus 已提交
953 954
            UInt64 scale = extractToDecimalScale(arguments[1]);

955
            if constexpr (std::is_same_v<Name, NameToDecimal32>)
956
                return createDecimal<DataTypeDecimal>(9, scale);
957
            else if constexpr (std::is_same_v<Name, NameToDecimal64>)
958
                return createDecimal<DataTypeDecimal>(18, scale);
959
            else if constexpr (std::is_same_v<Name, NameToDecimal128>)
960
                return createDecimal<DataTypeDecimal>(38, scale);
C
chertus 已提交
961 962 963

            throw Exception("Someting wrong with toDecimalNN()", ErrorCodes::LOGICAL_ERROR);
        }
A
Alexey Milovidov 已提交
964 965
        else
        {
V
Vasily Nemkov 已提交
966
            // Optional second argument with time zone for DateTime.
967
            UInt8 timezone_arg_position = 1;
968
            UInt32 scale [[maybe_unused]] = DataTypeDateTime64::default_scale;
969

970
            // DateTime64 requires more arguments: scale and timezone. Since timezone is optional, scale should be first.
971
            if constexpr (to_datetime64)
972
            {
973 974
                timezone_arg_position += 1;
                scale = static_cast<UInt32>(arguments[1].column->get64(0));
975 976
            }

977 978
            if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
                return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0));
979
            else if constexpr (to_datetime64)
980
                return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, timezone_arg_position, 0));
A
Alexey Milovidov 已提交
981
            else
N
Nikolai Kochetov 已提交
982
                return std::make_shared<ToDataType>();
A
Alexey Milovidov 已提交
983
        }
984 985
    }

986 987
    bool useDefaultImplementationForConstants() const override { return true; }
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
988
    bool canBeExecutedOnDefaultArguments() const override { return false; }
989

T
Tsarkova Anastasia 已提交
990
    void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
991 992 993
    {
        try
        {
T
Tsarkova Anastasia 已提交
994
            executeInternal(block, arguments, result, input_rows_count);
995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011
        }
        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
1012 1013
                || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
                || e.code() == ErrorCodes::CANNOT_PARSE_UUID)
1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032
            {
                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);
    }
1033 1034

private:
T
Tsarkova Anastasia 已提交
1035
    void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count)
1036
    {
1037 1038
        if (!arguments.size())
            throw Exception{"Function " + getName() + " expects at least 1 arguments",
A
Alexey Milovidov 已提交
1039
               ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
1040

1041
        const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
1042

1043 1044 1045 1046 1047 1048
        auto call = [&](const auto & types) -> bool
        {
            using Types = std::decay_t<decltype(types)>;
            using LeftDataType = typename Types::LeftType;
            using RightDataType = typename Types::RightType;

1049
            if constexpr (IsDataTypeDecimal<RightDataType>)
C
chertus 已提交
1050
            {
1051 1052 1053 1054 1055 1056 1057 1058 1059
                if constexpr (std::is_same_v<RightDataType, DataTypeDateTime64>)
                {
                    // account for optional timezone argument
                    if (arguments.size() != 2 && arguments.size() != 3)
                        throw Exception{"Function " + getName() + " expects 2 or 3 arguments for DataTypeDateTime64.",
                            ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
                }
                else if (arguments.size() != 2)
                {
C
chertus 已提交
1060
                    throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.",
A
Alexey Milovidov 已提交
1061
                        ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};
1062
                }
C
chertus 已提交
1063 1064 1065 1066 1067 1068

                const ColumnWithTypeAndName & scale_column = block.getByPosition(arguments[1]);
                UInt32 scale = extractToDecimalScale(scale_column);

                ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, scale);
            }
1069 1070 1071 1072 1073
            else if constexpr (IsDataTypeDateOrDateTime<RightDataType> && std::is_same_v<LeftDataType, DataTypeDateTime64>)
            {
                const auto * dt64 = assert_cast<const DataTypeDateTime64 *>(block.getByPosition(arguments[0]).type.get());
                ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count, dt64->getScale());
            }
C
chertus 已提交
1074 1075
            else
                ConvertImpl<LeftDataType, RightDataType, Name>::execute(block, arguments, result, input_rows_count);
1076

1077 1078 1079 1080 1081
            return true;
        };

        bool done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call);
        if (!done)
1082 1083
        {
            /// Generic conversion of any type to String.
A
Alexey Milovidov 已提交
1084
            if (std::is_same_v<ToDataType, DataTypeString>)
1085 1086 1087 1088
            {
                ConvertImplGenericToString::execute(block, arguments, result);
            }
            else
1089
                throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
1090 1091 1092
                    ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
        }
    }
A
Alexey Milovidov 已提交
1093 1094 1095
};


1096
/** Function toTOrZero (where T is number of date or datetime type):
1097 1098
  *  try to convert from String to type T through parsing,
  *  if cannot parse, return default value instead of throwing exception.
1099 1100
  * Function toTOrNull will return Nullable type with NULL when cannot parse.
  * NOTE Also need to implement tryToUnixTimestamp with timezone.
1101
  */
1102 1103 1104 1105
template <typename ToDataType, typename Name,
    ConvertFromStringExceptionMode exception_mode,
    ConvertFromStringParsingMode parsing_mode = ConvertFromStringParsingMode::Normal>
class FunctionConvertFromString : public IFunction
1106 1107
{
public:
1108
    static constexpr auto name = Name::name;
C
chertus 已提交
1109 1110 1111 1112 1113
    static constexpr bool to_decimal =
        std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
        std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
        std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>;

1114
    static FunctionPtr create(const Context &) { return std::make_shared<FunctionConvertFromString>(); }
1115
    static FunctionPtr create() { return std::make_shared<FunctionConvertFromString>(); }
1116 1117 1118 1119 1120 1121

    String getName() const override
    {
        return name;
    }

1122 1123
    bool isVariadic() const override { return true; }
    size_t getNumberOfArguments() const override { return 0; }
1124

1125
    bool useDefaultImplementationForConstants() const override { return true; }
1126
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
1127

1128
    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
1129
    {
1130 1131 1132
        if ((arguments.size() != 1 && arguments.size() != 2) || (to_decimal && arguments.size() != 2))
            throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) +
                ", should be 1 or 2. Second argument only make sense for DateTime (time zone, optional) and Decimal (scale).",
1133
                ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
1134

1135
        if (!isStringOrFixedString(arguments[0].type))
1136 1137 1138
        {
            if (this->getName().find("OrZero") != std::string::npos ||
                this->getName().find("OrNull") != std::string::npos)
1139
                throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
1140 1141 1142
                        ". Conversion functions with postfix 'OrZero' or 'OrNull'  should take String argument",
                        ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
            else
1143
                throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
1144 1145
                        ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
        }
1146

1147 1148
        if (arguments.size() == 2)
        {
1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164
            if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
            {
                if (!isString(arguments[1].type))
                    throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
                        ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
            }
            else if constexpr (to_decimal)
            {
                if (!isInteger(arguments[1].type))
                    throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
                        ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
                if (!arguments[1].column)
                    throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
            }
            else
            {
1165
                throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
1166
                    + toString(arguments.size()) + ", should be 1. Second argument makes sense only for DateTime and Decimal.",
1167
                    ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
1168
            }
1169
        }
1170

A
Alexey Milovidov 已提交
1171 1172
        DataTypePtr res;

C
chertus 已提交
1173
        if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
A
Alexey Milovidov 已提交
1174
            res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0));
C
chertus 已提交
1175
        else if constexpr (to_decimal)
1176 1177 1178 1179
        {
            UInt64 scale = extractToDecimalScale(arguments[1]);

            if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>>)
1180
                res = createDecimal<DataTypeDecimal>(9, scale);
1181
            else if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>>)
1182
                res = createDecimal<DataTypeDecimal>(18, scale);
1183
            else if constexpr (std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
1184
                res = createDecimal<DataTypeDecimal>(38, scale);
1185 1186 1187 1188

            if (!res)
                throw Exception("Someting wrong with toDecimalNNOrZero() or toDecimalNNOrNull()", ErrorCodes::LOGICAL_ERROR);
        }
V
Vasily Nemkov 已提交
1189 1190
        else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
        {
V
Vasily Nemkov 已提交
1191
            UInt64 scale = DataTypeDateTime64::default_scale;
N
Nikolai Kochetov 已提交
1192
            if (arguments.size() > 1)
V
Vasily Nemkov 已提交
1193
                scale = extractToDecimalScale(arguments[1]);
V
Vasily Nemkov 已提交
1194 1195 1196
            const auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0);
            res = std::make_shared<DataTypeDateTime64>(scale, timezone);
        }
1197
        else
A
Alexey Milovidov 已提交
1198 1199 1200 1201 1202 1203
            res = std::make_shared<ToDataType>();

        if constexpr (exception_mode == ConvertFromStringExceptionMode::Null)
            res = std::make_shared<DataTypeNullable>(res);

        return res;
1204 1205
    }

T
Tsarkova Anastasia 已提交
1206
    void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
1207 1208 1209
    {
        const IDataType * from_type = block.getByPosition(arguments[0]).type.get();

1210
        bool ok = true;
V
Vasily Nemkov 已提交
1211
        if constexpr (to_decimal || std::is_same_v<ToDataType, DataTypeDateTime64>)
1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230
        {
            if (arguments.size() != 2)
                throw Exception{"Function " + getName() + " expects 2 arguments for Decimal.", ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION};

            UInt32 scale = extractToDecimalScale(block.getByPosition(arguments[1]));

            if (checkAndGetDataType<DataTypeString>(from_type))
            {
                ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(
                    block, arguments, result, input_rows_count, scale);
            }
            else if (checkAndGetDataType<DataTypeFixedString>(from_type))
            {
                ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(
                    block, arguments, result, input_rows_count, scale);
            }
            else
                ok = false;
        }
1231
        else
1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248
        {
            if (checkAndGetDataType<DataTypeString>(from_type))
            {
                ConvertThroughParsing<DataTypeString, ToDataType, Name, exception_mode, parsing_mode>::execute(
                    block, arguments, result, input_rows_count);
            }
            else if (checkAndGetDataType<DataTypeFixedString>(from_type))
            {
                ConvertThroughParsing<DataTypeFixedString, ToDataType, Name, exception_mode, parsing_mode>::execute(
                    block, arguments, result, input_rows_count);
            }
            else
                ok = false;

        }

        if (!ok)
1249
            throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName()
C
chertus 已提交
1250 1251
                + ". Only String or FixedString argument is accepted for try-conversion function."
                + " For other arguments, use function without 'orZero' or 'orNull'.",
1252 1253
                ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
    }
1254 1255 1256
};


1257
/** Conversion to fixed string is implemented only for strings.
1258 1259 1260 1261
  */
class FunctionToFixedString : public IFunction
{
public:
1262
    static constexpr auto name = "toFixedString";
1263
    static FunctionPtr create(const Context &) { return std::make_shared<FunctionToFixedString>(); }
1264
    static FunctionPtr create() { return std::make_shared<FunctionToFixedString>(); }
1265 1266 1267 1268 1269 1270 1271 1272 1273

    String getName() const override
    {
        return name;
    }

    size_t getNumberOfArguments() const override { return 2; }
    bool isInjective(const Block &) override { return true; }

N
Nikolai Kochetov 已提交
1274
    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
1275
    {
1276
        if (!isUnsignedInteger(arguments[1].type))
1277
            throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
1278 1279
        if (!arguments[1].column)
            throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
1280
        if (!isStringOrFixedString(arguments[0].type))
1281 1282
            throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);

1283
        const size_t n = arguments[1].column->getUInt(0);
N
Nikolai Kochetov 已提交
1284
        return std::make_shared<DataTypeFixedString>(n);
1285 1286
    }

1287 1288 1289
    bool useDefaultImplementationForConstants() const override { return true; }
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }

T
Tsarkova Anastasia 已提交
1290
    void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
1291
    {
1292
        const auto n = block.getByPosition(arguments[1]).column->getUInt(0);
T
Tsarkova Anastasia 已提交
1293
        return executeForN(block, arguments, result, n);
1294 1295
    }

T
Tsarkova Anastasia 已提交
1296
    static void executeForN(Block & block, const ColumnNumbers & arguments, const size_t result, const size_t n)
1297
    {
1298
        const auto & column = block.getByPosition(arguments[0]).column;
1299

1300
        if (const auto column_string = checkAndGetColumn<ColumnString>(column.get()))
1301
        {
1302
            auto column_fixed = ColumnFixedString::create(n);
1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315 1316 1317 1318 1319

            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);
            }

1320
            block.getByPosition(result).column = std::move(column_fixed);
1321
        }
1322
        else if (const auto column_fixed_string = checkAndGetColumn<ColumnFixedString>(column.get()))
1323 1324 1325
        {
            const auto src_n = column_fixed_string->getN();
            if (src_n > n)
1326
                throw Exception{"String too long for type FixedString(" + toString(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE};
1327

1328
            auto column_fixed = ColumnFixedString::create(n);
1329 1330 1331 1332 1333 1334 1335 1336

            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);
1337 1338

            block.getByPosition(result).column = std::move(column_fixed);
1339 1340 1341 1342
        }
        else
            throw Exception("Unexpected column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
    }
1343 1344 1345
};


1346
/// Monotonicity.
1347

1348
struct PositiveMonotonicity
1349
{
1350
    static bool has() { return true; }
A
Alexey Milovidov 已提交
1351
    static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &)
1352 1353 1354
    {
        return { true };
    }
1355 1356
};

C
chertus 已提交
1357 1358 1359 1360 1361 1362 1363 1364 1365
struct UnknownMonotonicity
{
    static bool has() { return false; }
    static IFunction::Monotonicity get(const IDataType &, const Field &, const Field &)
    {
        return { false };
    }
};

1366
template <typename T>
D
dimarub2000 已提交
1367
struct ToNumberMonotonicity
1368
{
1369 1370
    static bool has() { return true; }

A
Alexey Milovidov 已提交
1371 1372 1373 1374 1375 1376 1377
    static UInt64 divideByRangeOfType(UInt64 x)
    {
        if constexpr (sizeof(T) < sizeof(UInt64))
            return x >> (sizeof(T) * 8);
        else
            return 0;
    }
1378 1379 1380

    static IFunction::Monotonicity get(const IDataType & type, const Field & left, const Field & right)
    {
Z
zhang2014 已提交
1381
        if (!type.isValueRepresentedByNumber())
1382 1383
            return {};

1384 1385
        /// If type is same, the conversion is always monotonic.
        /// (Enum has separate case, because it is different data type)
1386
        if (checkAndGetDataType<DataTypeNumber<T>>(&type) ||
1387
            checkAndGetDataType<DataTypeEnum<T>>(&type))
1388
            return { true, true, true };
1389

1390
        /// Float cases.
A
Alexey Milovidov 已提交
1391

1392 1393 1394 1395 1396
        /// When converting to Float, the conversion is always monotonic.
        if (std::is_floating_point_v<T>)
            return {true, true, true};

        /// If converting from Float, for monotonicity, arguments must fit in range of result type.
1397
        if (WhichDataType(type).isFloat())
1398
        {
1399 1400 1401
            if (left.isNull() || right.isNull())
                return {};

1402 1403 1404 1405 1406 1407 1408 1409 1410 1411
            Float64 left_float = left.get<Float64>();
            Float64 right_float = right.get<Float64>();

            if (left_float >= std::numeric_limits<T>::min() && left_float <= std::numeric_limits<T>::max()
                && right_float >= std::numeric_limits<T>::min() && right_float <= std::numeric_limits<T>::max())
                return { true };

            return {};
        }

1412 1413
        /// Integer cases.

A
Alexey Milovidov 已提交
1414
        const bool from_is_unsigned = type.isValueRepresentedByUnsignedInteger();
A
Amos Bird 已提交
1415
        const bool to_is_unsigned = is_unsigned_v<T>;
1416

A
Alexey Milovidov 已提交
1417 1418
        const size_t size_of_from = type.getSizeOfValueInMemory();
        const size_t size_of_to = sizeof(T);
1419

A
Alexey Milovidov 已提交
1420
        const bool left_in_first_half = left.isNull()
1421
            ? from_is_unsigned
A
Alexey Milovidov 已提交
1422
            : (left.get<Int64>() >= 0);
1423

A
Alexey Milovidov 已提交
1424
        const bool right_in_first_half = right.isNull()
1425
            ? !from_is_unsigned
A
Alexey Milovidov 已提交
1426
            : (right.get<Int64>() >= 0);
A
Alexey Milovidov 已提交
1427

1428 1429 1430 1431 1432 1433 1434 1435
        /// Size of type is the same.
        if (size_of_from == size_of_to)
        {
            if (from_is_unsigned == to_is_unsigned)
                return {true, true, true};

            if (left_in_first_half == right_in_first_half)
                return {true};
1436 1437

            return {};
1438
        }
1439

1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451
        /// Size of type is expanded.
        if (size_of_from < size_of_to)
        {
            if (from_is_unsigned == to_is_unsigned)
                return {true, true, true};

            if (!to_is_unsigned)
                return {true, true, true};

            /// signed -> unsigned. If arguments from the same half, then function is monotonic.
            if (left_in_first_half == right_in_first_half)
                return {true};
A
Alexey Milovidov 已提交
1452 1453

            return {};
1454 1455 1456 1457 1458 1459 1460 1461 1462 1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473
        }

        /// Size of type is shrinked.
        if (size_of_from > size_of_to)
        {
            /// Function cannot be monotonic on unbounded ranges.
            if (left.isNull() || right.isNull())
                return {};

            if (from_is_unsigned == to_is_unsigned)
            {
                /// all bits other than that fits, must be same.
                if (divideByRangeOfType(left.get<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
                    return {true};

                return {};
            }
            else
            {
                /// When signedness is changed, it's also required for arguments to be from the same half.
A
Alexey Milovidov 已提交
1474
                /// And they must be in the same half after converting to the result type.
1475
                if (left_in_first_half == right_in_first_half
A
Alexey Milovidov 已提交
1476
                    && (T(left.get<Int64>()) >= 0) == (T(right.get<Int64>()) >= 0)
1477 1478 1479 1480 1481 1482 1483 1484
                    && divideByRangeOfType(left.get<UInt64>()) == divideByRangeOfType(right.get<UInt64>()))
                    return {true};

                return {};
            }
        }

        __builtin_unreachable();
1485
    }
1486 1487
};

F
f1yegor 已提交
1488 1489
/** 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`.
1490 1491
  */
struct ToStringMonotonicity
1492
{
1493 1494 1495 1496 1497 1498 1499
    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;

F
f1yegor 已提交
1500
        /// `toString` function is monotonous if the argument is Date or DateTime, or non-negative numbers with the same number of symbols.
1501

1502
        if (checkAndGetDataType<DataTypeDate>(&type)
1503 1504 1505 1506 1507 1508 1509 1510 1511 1512 1513 1514 1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526
            || typeid_cast<const DataTypeDateTime *>(&type))
            return positive;

        if (left.isNull() || right.isNull())
            return {};

        if (left.getType() == Field::Types::UInt64
            && right.getType() == Field::Types::UInt64)
        {
            return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
                || (floor(log10(left.get<UInt64>())) == floor(log10(right.get<UInt64>())))
                ? positive : not_monotonic;
        }

        if (left.getType() == Field::Types::Int64
            && right.getType() == Field::Types::Int64)
        {
            return (left.get<Int64>() == 0 && right.get<Int64>() == 0)
                || (left.get<Int64>() > 0 && right.get<Int64>() > 0 && floor(log10(left.get<Int64>())) == floor(log10(right.get<Int64>())))
                ? positive : not_monotonic;
        }

        return not_monotonic;
    }
1527 1528
};

1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541

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"; };

D
dimarub2000 已提交
1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553
using FunctionToUInt8 = FunctionConvert<DataTypeUInt8, NameToUInt8, ToNumberMonotonicity<UInt8>>;
using FunctionToUInt16 = FunctionConvert<DataTypeUInt16, NameToUInt16, ToNumberMonotonicity<UInt16>>;
using FunctionToUInt32 = FunctionConvert<DataTypeUInt32, NameToUInt32, ToNumberMonotonicity<UInt32>>;
using FunctionToUInt64 = FunctionConvert<DataTypeUInt64, NameToUInt64, ToNumberMonotonicity<UInt64>>;
using FunctionToInt8 = FunctionConvert<DataTypeInt8, NameToInt8, ToNumberMonotonicity<Int8>>;
using FunctionToInt16 = FunctionConvert<DataTypeInt16, NameToInt16, ToNumberMonotonicity<Int16>>;
using FunctionToInt32 = FunctionConvert<DataTypeInt32, NameToInt32, ToNumberMonotonicity<Int32>>;
using FunctionToInt64 = FunctionConvert<DataTypeInt64, NameToInt64, ToNumberMonotonicity<Int64>>;
using FunctionToFloat32 = FunctionConvert<DataTypeFloat32, NameToFloat32, ToNumberMonotonicity<Float32>>;
using FunctionToFloat64 = FunctionConvert<DataTypeFloat64, NameToFloat64, ToNumberMonotonicity<Float64>>;
using FunctionToDate = FunctionConvert<DataTypeDate, NameToDate, ToNumberMonotonicity<UInt16>>;
using FunctionToDateTime = FunctionConvert<DataTypeDateTime, NameToDateTime, ToNumberMonotonicity<UInt32>>;
V
Vasily Nemkov 已提交
1554
using FunctionToDateTime64 = FunctionConvert<DataTypeDateTime64, NameToDateTime64, UnknownMonotonicity>;
D
dimarub2000 已提交
1555
using FunctionToUUID = FunctionConvert<DataTypeUUID, NameToUUID, ToNumberMonotonicity<UInt128>>;
1556
using FunctionToString = FunctionConvert<DataTypeString, NameToString, ToStringMonotonicity>;
D
dimarub2000 已提交
1557
using FunctionToUnixTimestamp = FunctionConvert<DataTypeUInt32, NameToUnixTimestamp, ToNumberMonotonicity<UInt32>>;
C
chertus 已提交
1558 1559 1560
using FunctionToDecimal32 = FunctionConvert<DataTypeDecimal<Decimal32>, NameToDecimal32, UnknownMonotonicity>;
using FunctionToDecimal64 = FunctionConvert<DataTypeDecimal<Decimal64>, NameToDecimal64, UnknownMonotonicity>;
using FunctionToDecimal128 = FunctionConvert<DataTypeDecimal<Decimal128>, NameToDecimal128, UnknownMonotonicity>;
1561

A
Alexey Milovidov 已提交
1562

1563
template <typename DataType> struct FunctionTo;
1564

1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576
template <> struct FunctionTo<DataTypeUInt8> { using Type = FunctionToUInt8; };
template <> struct FunctionTo<DataTypeUInt16> { using Type = FunctionToUInt16; };
template <> struct FunctionTo<DataTypeUInt32> { using Type = FunctionToUInt32; };
template <> struct FunctionTo<DataTypeUInt64> { using Type = FunctionToUInt64; };
template <> struct FunctionTo<DataTypeInt8> { using Type = FunctionToInt8; };
template <> struct FunctionTo<DataTypeInt16> { using Type = FunctionToInt16; };
template <> struct FunctionTo<DataTypeInt32> { using Type = FunctionToInt32; };
template <> struct FunctionTo<DataTypeInt64> { using Type = FunctionToInt64; };
template <> struct FunctionTo<DataTypeFloat32> { using Type = FunctionToFloat32; };
template <> struct FunctionTo<DataTypeFloat64> { using Type = FunctionToFloat64; };
template <> struct FunctionTo<DataTypeDate> { using Type = FunctionToDate; };
template <> struct FunctionTo<DataTypeDateTime> { using Type = FunctionToDateTime; };
1577
template <> struct FunctionTo<DataTypeDateTime64> { using Type = FunctionToDateTime64; };
1578
template <> struct FunctionTo<DataTypeUUID> { using Type = FunctionToUUID; };
1579 1580
template <> struct FunctionTo<DataTypeString> { using Type = FunctionToString; };
template <> struct FunctionTo<DataTypeFixedString> { using Type = FunctionToFixedString; };
1581 1582 1583
template <> struct FunctionTo<DataTypeDecimal<Decimal32>> { using Type = FunctionToDecimal32; };
template <> struct FunctionTo<DataTypeDecimal<Decimal64>> { using Type = FunctionToDecimal64; };
template <> struct FunctionTo<DataTypeDecimal<Decimal128>> { using Type = FunctionToDecimal128; };
1584

1585
template <typename FieldType> struct FunctionTo<DataTypeEnum<FieldType>>
1586
    : FunctionTo<DataTypeNumber<FieldType>>
1587 1588
{
};
1589

1590 1591 1592 1593 1594 1595 1596 1597 1598 1599
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"; };
1600 1601
struct NameToDateOrZero { static constexpr auto name = "toDateOrZero"; };
struct NameToDateTimeOrZero { static constexpr auto name = "toDateTimeOrZero"; };
1602
struct NameToDateTime64OrZero { static constexpr auto name = "toDateTime64OrZero"; };
1603 1604 1605
struct NameToDecimal32OrZero { static constexpr auto name = "toDecimal32OrZero"; };
struct NameToDecimal64OrZero { static constexpr auto name = "toDecimal64OrZero"; };
struct NameToDecimal128OrZero { static constexpr auto name = "toDecimal128OrZero"; };
1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618

using FunctionToUInt8OrZero = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToUInt16OrZero = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToUInt32OrZero = FunctionConvertFromString<DataTypeUInt32, NameToUInt32OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToUInt64OrZero = FunctionConvertFromString<DataTypeUInt64, NameToUInt64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToInt8OrZero = FunctionConvertFromString<DataTypeInt8, NameToInt8OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToInt16OrZero = FunctionConvertFromString<DataTypeInt16, NameToInt16OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToInt32OrZero = FunctionConvertFromString<DataTypeInt32, NameToInt32OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToInt64OrZero = FunctionConvertFromString<DataTypeInt64, NameToInt64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToFloat32OrZero = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToFloat64OrZero = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDateOrZero = FunctionConvertFromString<DataTypeDate, NameToDateOrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDateTimeOrZero = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrZero, ConvertFromStringExceptionMode::Zero>;
1619
using FunctionToDateTime64OrZero = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrZero, ConvertFromStringExceptionMode::Zero>;
1620 1621 1622
using FunctionToDecimal32OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDecimal64OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal64>, NameToDecimal64OrZero, ConvertFromStringExceptionMode::Zero>;
using FunctionToDecimal128OrZero = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrZero, ConvertFromStringExceptionMode::Zero>;
1623

1624 1625 1626 1627 1628 1629 1630 1631 1632 1633
struct NameToUInt8OrNull { static constexpr auto name = "toUInt8OrNull"; };
struct NameToUInt16OrNull { static constexpr auto name = "toUInt16OrNull"; };
struct NameToUInt32OrNull { static constexpr auto name = "toUInt32OrNull"; };
struct NameToUInt64OrNull { static constexpr auto name = "toUInt64OrNull"; };
struct NameToInt8OrNull { static constexpr auto name = "toInt8OrNull"; };
struct NameToInt16OrNull { static constexpr auto name = "toInt16OrNull"; };
struct NameToInt32OrNull { static constexpr auto name = "toInt32OrNull"; };
struct NameToInt64OrNull { static constexpr auto name = "toInt64OrNull"; };
struct NameToFloat32OrNull { static constexpr auto name = "toFloat32OrNull"; };
struct NameToFloat64OrNull { static constexpr auto name = "toFloat64OrNull"; };
1634 1635
struct NameToDateOrNull { static constexpr auto name = "toDateOrNull"; };
struct NameToDateTimeOrNull { static constexpr auto name = "toDateTimeOrNull"; };
1636
struct NameToDateTime64OrNull { static constexpr auto name = "toDateTime64OrNull"; };
1637 1638 1639
struct NameToDecimal32OrNull { static constexpr auto name = "toDecimal32OrNull"; };
struct NameToDecimal64OrNull { static constexpr auto name = "toDecimal64OrNull"; };
struct NameToDecimal128OrNull { static constexpr auto name = "toDecimal128OrNull"; };
1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652

using FunctionToUInt8OrNull = FunctionConvertFromString<DataTypeUInt8, NameToUInt8OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToUInt16OrNull = FunctionConvertFromString<DataTypeUInt16, NameToUInt16OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToUInt32OrNull = FunctionConvertFromString<DataTypeUInt32, NameToUInt32OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToUInt64OrNull = FunctionConvertFromString<DataTypeUInt64, NameToUInt64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToInt8OrNull = FunctionConvertFromString<DataTypeInt8, NameToInt8OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToInt16OrNull = FunctionConvertFromString<DataTypeInt16, NameToInt16OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToInt32OrNull = FunctionConvertFromString<DataTypeInt32, NameToInt32OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToInt64OrNull = FunctionConvertFromString<DataTypeInt64, NameToInt64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToFloat32OrNull = FunctionConvertFromString<DataTypeFloat32, NameToFloat32OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToFloat64OrNull = FunctionConvertFromString<DataTypeFloat64, NameToFloat64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDateOrNull = FunctionConvertFromString<DataTypeDate, NameToDateOrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDateTimeOrNull = FunctionConvertFromString<DataTypeDateTime, NameToDateTimeOrNull, ConvertFromStringExceptionMode::Null>;
1653
using FunctionToDateTime64OrNull = FunctionConvertFromString<DataTypeDateTime64, NameToDateTime64OrNull, ConvertFromStringExceptionMode::Null>;
1654 1655 1656
using FunctionToDecimal32OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal32>, NameToDecimal32OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDecimal64OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal64>, NameToDecimal64OrNull, ConvertFromStringExceptionMode::Null>;
using FunctionToDecimal128OrNull = FunctionConvertFromString<DataTypeDecimal<Decimal128>, NameToDecimal128OrNull, ConvertFromStringExceptionMode::Null>;
1657 1658 1659 1660

struct NameParseDateTimeBestEffort { static constexpr auto name = "parseDateTimeBestEffort"; };
struct NameParseDateTimeBestEffortOrZero { static constexpr auto name = "parseDateTimeBestEffortOrZero"; };
struct NameParseDateTimeBestEffortOrNull { static constexpr auto name = "parseDateTimeBestEffortOrNull"; };
1661 1662 1663 1664
struct NameParseDateTime64BestEffort { static constexpr auto name = "parseDateTime64BestEffort"; };
struct NameParseDateTime64BestEffortOrZero { static constexpr auto name = "parseDateTime64BestEffortOrZero"; };
struct NameParseDateTime64BestEffortOrNull { static constexpr auto name = "parseDateTime64BestEffortOrNull"; };

1665 1666 1667 1668 1669 1670 1671

using FunctionParseDateTimeBestEffort = FunctionConvertFromString<
    DataTypeDateTime, NameParseDateTimeBestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTimeBestEffortOrZero = FunctionConvertFromString<
    DataTypeDateTime, NameParseDateTimeBestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTimeBestEffortOrNull = FunctionConvertFromString<
    DataTypeDateTime, NameParseDateTimeBestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
1672

1673 1674 1675 1676 1677 1678
using FunctionParseDateTime64BestEffort = FunctionConvertFromString<
    DataTypeDateTime64, NameParseDateTime64BestEffort, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime64BestEffortOrZero = FunctionConvertFromString<
    DataTypeDateTime64, NameParseDateTime64BestEffortOrZero, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::BestEffort>;
using FunctionParseDateTime64BestEffortOrNull = FunctionConvertFromString<
    DataTypeDateTime64, NameParseDateTime64BestEffortOrNull, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::BestEffort>;
1679

N
Nikolai Kochetov 已提交
1680
class ExecutableFunctionCast : public IExecutableFunctionImpl
1681
{
1682
public:
T
Tsarkova Anastasia 已提交
1683
    using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
N
Nikolai Kochetov 已提交
1684

N
Nikolai Kochetov 已提交
1685
    explicit ExecutableFunctionCast(WrapperType && wrapper_function_, const char * name_)
K
kreuzerkrieg 已提交
1686
            : wrapper_function(std::move(wrapper_function_)), name(name_) {}
N
Nikolai Kochetov 已提交
1687 1688 1689 1690

    String getName() const override { return name; }

protected:
N
Nikolai Kochetov 已提交
1691
    void execute(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
N
Nikolai Kochetov 已提交
1692 1693 1694 1695 1696 1697
    {
        /// 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));

T
Tsarkova Anastasia 已提交
1698
        wrapper_function(block, new_arguments, result, input_rows_count);
N
Nikolai Kochetov 已提交
1699 1700 1701 1702
    }

    bool useDefaultImplementationForNulls() const override { return false; }
    bool useDefaultImplementationForConstants() const override { return true; }
1703
    bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
N
Nikolai Kochetov 已提交
1704
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
1705 1706

private:
N
Nikolai Kochetov 已提交
1707 1708 1709 1710
    WrapperType wrapper_function;
    const char * name;
};

1711 1712 1713

struct NameCast { static constexpr auto name = "CAST"; };

N
Nikolai Kochetov 已提交
1714
class FunctionCast final : public IFunctionBaseImpl
N
Nikolai Kochetov 已提交
1715 1716
{
public:
T
Tsarkova Anastasia 已提交
1717
    using WrapperType = std::function<void(Block &, const ColumnNumbers &, size_t, size_t)>;
1718
    using MonotonicityForRange = std::function<Monotonicity(const IDataType &, const Field &, const Field &)>;
N
Nikolai Kochetov 已提交
1719

1720
    FunctionCast(const char * name_, MonotonicityForRange && monotonicity_for_range_
K
kreuzerkrieg 已提交
1721
            , const DataTypes & argument_types_, const DataTypePtr & return_type_)
1722
            : name(name_), monotonicity_for_range(monotonicity_for_range_)
K
kreuzerkrieg 已提交
1723
            , argument_types(argument_types_), return_type(return_type_)
N
Nikolai Kochetov 已提交
1724 1725 1726 1727 1728 1729
    {
    }

    const DataTypes & getArgumentTypes() const override { return argument_types; }
    const DataTypePtr & getReturnType() const override { return return_type; }

N
Nikolai Kochetov 已提交
1730
    ExecutableFunctionImplPtr prepare(const Block & /*sample_block*/, const ColumnNumbers & /*arguments*/, size_t /*result*/) const override
N
Nikolai Kochetov 已提交
1731
    {
N
Nikolai Kochetov 已提交
1732
        return std::make_unique<ExecutableFunctionCast>(
1733
                prepareUnpackDictionaries(getArgumentTypes()[0], getReturnType()), name);
N
Nikolai Kochetov 已提交
1734 1735 1736 1737
    }

    String getName() const override { return name; }

1738 1739 1740
    bool isDeterministic() const override { return true; }
    bool isDeterministicInScopeOfQuery() const override { return true; }

N
Nikolai Kochetov 已提交
1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757
    bool hasInformationAboutMonotonicity() const override
    {
        return static_cast<bool>(monotonicity_for_range);
    }

    Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
    {
        return monotonicity_for_range(type, left, right);
    }

private:

    const char * name;
    MonotonicityForRange monotonicity_for_range;

    DataTypes argument_types;
    DataTypePtr return_type;
1758

1759
    template <typename DataType>
1760
    WrapperType createWrapper(const DataTypePtr & from_type, const DataType * const, bool requested_result_is_nullable) const
1761
    {
1762
        FunctionPtr function;
1763

1764 1765 1766 1767
        if (requested_result_is_nullable && checkAndGetDataType<DataTypeString>(from_type.get()))
        {
            /// In case when converting to Nullable type, we apply different parsing rule,
            /// that will not throw an exception but return NULL in case of malformed input.
1768
            function = FunctionConvertFromString<DataType, NameCast, ConvertFromStringExceptionMode::Null>::create();
1769 1770
        }
        else
1771
            function = FunctionTo<DataType>::Type::create();
1772

N
Nikolai Kochetov 已提交
1773
        auto function_adaptor =
1774
                FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
N
Nikolai Kochetov 已提交
1775
                .build({ColumnWithTypeAndName{nullptr, from_type, ""}});
1776

N
Nikolai Kochetov 已提交
1777
        return [function_adaptor] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
1778
        {
N
Nikolai Kochetov 已提交
1779
            function_adaptor->execute(block, arguments, result, input_rows_count);
1780 1781 1782 1783 1784
        };
    }

    WrapperType createStringWrapper(const DataTypePtr & from_type) const
    {
1785
        FunctionPtr function = FunctionToString::create();
1786

N
Nikolai Kochetov 已提交
1787
        auto function_adaptor =
1788
                FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
N
Nikolai Kochetov 已提交
1789
                .build({ColumnWithTypeAndName{nullptr, from_type, ""}});
1790

N
Nikolai Kochetov 已提交
1791
        return [function_adaptor] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
1792
        {
N
Nikolai Kochetov 已提交
1793
            function_adaptor->execute(block, arguments, result, input_rows_count);
1794 1795 1796
        };
    }

1797
    static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N)
1798
    {
1799
        if (!isStringOrFixedString(from_type))
1800
            throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED};
1801

T
Tsarkova Anastasia 已提交
1802
        return [N] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
1803
        {
T
Tsarkova Anastasia 已提交
1804
            FunctionToFixedString::executeForN(block, arguments, result, N);
1805 1806 1807
        };
    }

1808 1809 1810 1811 1812
    WrapperType createUUIDWrapper(const DataTypePtr & from_type, const DataTypeUUID * const, bool requested_result_is_nullable) const
    {
        if (requested_result_is_nullable)
            throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED};

1813
        FunctionPtr function = FunctionTo<DataTypeUUID>::Type::create();
1814

N
Nikolai Kochetov 已提交
1815
        auto function_adaptor =
1816
                FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
N
Nikolai Kochetov 已提交
1817
                .build({ColumnWithTypeAndName{nullptr, from_type, ""}});
1818

N
Nikolai Kochetov 已提交
1819
        return [function_adaptor] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
1820
        {
N
Nikolai Kochetov 已提交
1821
            function_adaptor->execute(block, arguments, result, input_rows_count);
1822 1823 1824
        };
    }

1825 1826 1827
    template <typename ToDataType>
    std::enable_if_t<IsDataTypeDecimal<ToDataType>, WrapperType>
    createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type) const
C
chertus 已提交
1828 1829 1830 1831
    {
        TypeIndex type_index = from_type->getTypeId();
        UInt32 scale = to_type->getScale();

1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842
        WhichDataType which(type_index);
        bool ok = which.isNativeInt() ||
            which.isNativeUInt() ||
            which.isDecimal() ||
            which.isFloat() ||
            which.isDateOrDateTime() ||
            which.isStringOrFixedString();
        if (!ok)
            throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
                ErrorCodes::CANNOT_CONVERT_TYPE};

1843
        return [type_index, scale, to_type] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
C
chertus 已提交
1844
        {
1845
            auto res = callOnIndexAndDataType<ToDataType>(type_index, [&](const auto & types) -> bool
C
chertus 已提交
1846 1847 1848 1849 1850 1851 1852 1853
            {
                using Types = std::decay_t<decltype(types)>;
                using LeftDataType = typename Types::LeftType;
                using RightDataType = typename Types::RightType;

                ConvertImpl<LeftDataType, RightDataType, NameCast>::execute(block, arguments, result, input_rows_count, scale);
                return true;
            });
1854 1855 1856 1857

            /// Additionally check if callOnIndexAndDataType wasn't called at all.
            if (!res)
            {
1858
                throw Exception{"Conversion from " + std::string(getTypeName(type_index)) + " to " + to_type->getName() +
1859 1860
                                " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
            }
C
chertus 已提交
1861 1862
        };
    }
1863

1864 1865 1866 1867 1868 1869 1870 1871 1872 1873 1874 1875 1876 1877
    WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type) const
    {
        /// Conversion from String through parsing.
        if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
        {
            return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
            {
                ConvertImplGenericFromString::execute(block, arguments, result);
            };
        }
        else
            throw Exception{"Conversion from " + from_type_untyped->getName() + " to " + to_type->getName() +
                " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
    }
C
chertus 已提交
1878

N
Nikolai Kochetov 已提交
1879
    WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) const
1880 1881
    {
        /// Conversion from String through parsing.
1882
        if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
1883
        {
T
Tsarkova Anastasia 已提交
1884
            return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
1885 1886 1887 1888 1889
            {
                ConvertImplGenericFromString::execute(block, arguments, result);
            };
        }

1890 1891
        DataTypePtr from_nested_type;
        DataTypePtr to_nested_type;
1892
        auto from_type = checkAndGetDataType<DataTypeArray>(from_type_untyped.get());
1893 1894

        /// get the most nested type
1895
        if (from_type && to_type)
1896 1897 1898 1899
        {
            from_nested_type = from_type->getNestedType();
            to_nested_type = to_type->getNestedType();

1900 1901
            from_type = checkAndGetDataType<DataTypeArray>(from_nested_type.get());
            to_type = checkAndGetDataType<DataTypeArray>(to_nested_type.get());
1902 1903 1904
        }

        /// both from_type and to_type should be nullptr now is array types had same dimensions
1905
        if ((from_type == nullptr) != (to_type == nullptr))
C
chertus 已提交
1906 1907
            throw Exception{"CAST AS Array can only be performed between same-dimensional array types or from String",
                ErrorCodes::TYPE_MISMATCH};
1908 1909

        /// Prepare nested type conversion
1910
        const auto nested_function = prepareUnpackDictionaries(from_nested_type, to_nested_type);
1911

1912
        return [nested_function, from_nested_type, to_nested_type](
T
Tsarkova Anastasia 已提交
1913
            Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
1914
        {
1915
            const auto & array_arg = block.getByPosition(arguments.front());
1916

1917
            if (const ColumnArray * col_array = checkAndGetColumn<ColumnArray>(array_arg.column.get()))
1918 1919
            {
                /// create block for converting nested column containing original and result columns
1920 1921
                Block nested_block
                {
1922 1923 1924 1925 1926
                    { col_array->getDataPtr(), from_nested_type, "" },
                    { nullptr, to_nested_type, "" }
                };

                /// convert nested column
T
Tsarkova Anastasia 已提交
1927
                nested_function(nested_block, {0}, 1, nested_block.rows());
1928 1929

                /// set converted nested column to result
1930
                block.getByPosition(result).column = ColumnArray::create(nested_block.getByPosition(1).column, col_array->getOffsetsPtr());
1931 1932
            }
            else
1933
                throw Exception{"Illegal column " + array_arg.column->getName() + " for function CAST AS Array", ErrorCodes::LOGICAL_ERROR};
1934 1935 1936
        };
    }

N
Nikolai Kochetov 已提交
1937
    WrapperType createTupleWrapper(const DataTypePtr & from_type_untyped, const DataTypeTuple * to_type) const
1938 1939
    {
        /// Conversion from String through parsing.
1940
        if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
1941
        {
T
Tsarkova Anastasia 已提交
1942
            return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
1943 1944 1945 1946 1947
            {
                ConvertImplGenericFromString::execute(block, arguments, result);
            };
        }

1948
        const auto from_type = checkAndGetDataType<DataTypeTuple>(from_type_untyped.get());
1949
        if (!from_type)
C
chertus 已提交
1950 1951
            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};
1952 1953

        if (from_type->getElements().size() != to_type->getElements().size())
1954 1955
            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};
1956 1957 1958 1959 1960 1961 1962

        const auto & from_element_types = from_type->getElements();
        const auto & to_element_types = to_type->getElements();
        std::vector<WrapperType> element_wrappers;
        element_wrappers.reserve(from_element_types.size());

        /// Create conversion wrapper for each element in tuple
1963
        for (const auto idx_type : ext::enumerate(from_type->getElements()))
1964
            element_wrappers.push_back(prepareUnpackDictionaries(idx_type.second, to_element_types[idx_type.first]));
1965

1966
        return [element_wrappers, from_element_types, to_element_types]
T
Tsarkova Anastasia 已提交
1967
            (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
1968
        {
1969
            const auto col = block.getByPosition(arguments.front()).column.get();
1970 1971 1972 1973

            /// copy tuple elements to a separate block
            Block element_block;

1974 1975 1976 1977 1978 1979
            size_t tuple_size = from_element_types.size();
            const ColumnTuple & column_tuple = typeid_cast<const ColumnTuple &>(*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], "" });
1980 1981 1982 1983 1984 1985 1986 1987 1988

            /// 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<DataTypeTuple>(to_element_types), "" });

            /// invoke conversion for each element
1989
            for (const auto idx_element_wrapper : ext::enumerate(element_wrappers))
1990
                idx_element_wrapper.second(element_block, { idx_element_wrapper.first },
T
Tsarkova Anastasia 已提交
1991
                    tuple_size + idx_element_wrapper.first, input_rows_count);
1992

1993 1994 1995
            Columns converted_columns(tuple_size);
            for (size_t i = 0; i < tuple_size; ++i)
                converted_columns[i] = element_block.getByPosition(tuple_size + i).column;
1996

1997
            block.getByPosition(result).column = ColumnTuple::create(converted_columns);
1998 1999 2000 2001
        };
    }

    template <typename FieldType>
N
Nikolai Kochetov 已提交
2002
    WrapperType createEnumWrapper(const DataTypePtr & from_type, const DataTypeEnum<FieldType> * to_type) const
2003 2004 2005 2006
    {
        using EnumType = DataTypeEnum<FieldType>;
        using Function = typename FunctionTo<EnumType>::Type;

2007
        if (const auto from_enum8 = checkAndGetDataType<DataTypeEnum8>(from_type.get()))
2008
            checkEnumToEnumConversion(from_enum8, to_type);
2009
        else if (const auto from_enum16 = checkAndGetDataType<DataTypeEnum16>(from_type.get()))
2010 2011
            checkEnumToEnumConversion(from_enum16, to_type);

2012
        if (checkAndGetDataType<DataTypeString>(from_type.get()))
2013
            return createStringToEnumWrapper<ColumnString, EnumType>();
2014
        else if (checkAndGetDataType<DataTypeFixedString>(from_type.get()))
2015
            return createStringToEnumWrapper<ColumnFixedString, EnumType>();
C
chertus 已提交
2016
        else if (isNativeNumber(from_type) || isEnum(from_type))
2017
        {
2018
            auto function = Function::create();
2019
            auto func_or_adaptor = FunctionOverloadResolverAdaptor(std::make_unique<DefaultOverloadResolver>(function))
N
Nikolai Kochetov 已提交
2020
                    .build(ColumnsWithTypeAndName{{nullptr, from_type, "" }});
2021

N
Nikolai Kochetov 已提交
2022
            return [func_or_adaptor] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
2023
            {
N
Nikolai Kochetov 已提交
2024
                func_or_adaptor->execute(block, arguments, result, input_rows_count);
2025 2026 2027
            };
        }
        else
2028 2029
            throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() +
                " is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
2030 2031 2032
    }

    template <typename EnumTypeFrom, typename EnumTypeTo>
N
Nikolai Kochetov 已提交
2033
    void checkEnumToEnumConversion(const EnumTypeFrom * from_type, const EnumTypeTo * to_type) const
2034 2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051
    {
        const auto & from_values = from_type->getValues();
        const auto & to_values = to_type->getValues();

        using ValueType = std::common_type_t<typename EnumTypeFrom::FieldType, typename EnumTypeTo::FieldType>;
        using NameValuePair = std::pair<std::string, ValueType>;
        using EnumValues = std::vector<NameValuePair>;

        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)
2052 2053
                throw Exception{"Enum conversion changes value for element '" + name_value.first +
                    "' from " + toString(old_value) + " to " + toString(new_value), ErrorCodes::CANNOT_CONVERT_TYPE};
2054
        }
2055
    }
2056 2057

    template <typename ColumnStringType, typename EnumType>
N
Nikolai Kochetov 已提交
2058
    WrapperType createStringToEnumWrapper() const
2059
    {
2060
        const char * function_name = name;
T
Tsarkova Anastasia 已提交
2061
        return [function_name] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
2062
        {
2063
            const auto first_col = block.getByPosition(arguments.front()).column.get();
2064

2065
            auto & col_with_type_and_name = block.getByPosition(result);
2066
            const auto & result_type = typeid_cast<const EnumType &>(*col_with_type_and_name.type);
2067 2068 2069 2070 2071 2072 2073 2074 2075 2076 2077 2078

            if (const auto col = typeid_cast<const ColumnStringType *>(first_col))
            {
                const auto size = col->size();

                auto res = result_type.createColumn();
                auto & out_data = static_cast<typename EnumType::ColumnType &>(*res).getData();
                out_data.resize(size);

                for (const auto i : ext::range(0, size))
                    out_data[i] = result_type.getValue(col->getDataAt(i));

2079
                col_with_type_and_name.column = std::move(res);
2080 2081
            }
            else
2082
                throw Exception{"Unexpected column " + first_col->getName() + " as first argument of function " + function_name,
2083
                    ErrorCodes::LOGICAL_ERROR};
2084 2085 2086
        };
    }

N
Nikolai Kochetov 已提交
2087
    WrapperType createIdentityWrapper(const DataTypePtr &) const
2088
    {
T
Tsarkova Anastasia 已提交
2089
        return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
2090
        {
2091
            block.getByPosition(result).column = block.getByPosition(arguments.front()).column;
2092 2093 2094
        };
    }

N
Nikolai Kochetov 已提交
2095
    WrapperType createNothingWrapper(const IDataType * to_type) const
2096
    {
A
Alexey Milovidov 已提交
2097
        ColumnPtr res = to_type->createColumnConstWithDefaultValue(1);
T
Tsarkova Anastasia 已提交
2098
        return [res] (Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
2099 2100
        {
            /// Column of Nothing type is trivially convertible to any other column
T
Tsarkova Anastasia 已提交
2101
            block.getByPosition(result).column = res->cloneResized(input_rows_count)->convertToFullColumnIfConst();
2102 2103 2104
        };
    }

2105
    WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const
2106
    {
2107 2108 2109 2110
        const auto * from_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(from_type.get());
        const auto * to_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(to_type.get());
        const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type;
        const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type;
2111

2112
        if (from_type->onlyNull())
2113
        {
2114
            if (!to_nested->isNullable())
2115 2116
                throw Exception{"Cannot convert NULL to a non-nullable type", ErrorCodes::CANNOT_CONVERT_TYPE};

T
Tsarkova Anastasia 已提交
2117
            return [](Block & block, const ColumnNumbers &, const size_t result, size_t input_rows_count)
2118 2119
            {
                auto & res = block.getByPosition(result);
T
Tsarkova Anastasia 已提交
2120
                res.column = res.type->createColumnConstWithDefaultValue(input_rows_count)->convertToFullColumnIfConst();
2121 2122 2123
            };
        }

2124 2125 2126 2127 2128 2129 2130
        bool skip_not_null_check = false;

        if (from_low_cardinality && from_nested->isNullable() && !to_nested->isNullable())
            /// Disable check for dictionary. Will check that column doesn't contain NULL in wrapper below.
            skip_not_null_check = true;

        auto wrapper = prepareRemoveNullable(from_nested, to_nested, skip_not_null_check);
2131
        if (!from_low_cardinality && !to_low_cardinality)
2132 2133
            return wrapper;

2134
        return [wrapper, from_low_cardinality, to_low_cardinality, skip_not_null_check]
2135 2136 2137 2138 2139 2140
                (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
        {
            auto & arg = block.getByPosition(arguments[0]);
            auto & res = block.getByPosition(result);

            ColumnPtr res_indexes;
2141 2142
            /// For some types default can't be casted (for example, String to Int). In that case convert column to full.
            bool src_converted_to_full_column = false;
2143 2144 2145 2146 2147 2148 2149 2150 2151 2152

            {
                /// Replace argument and result columns (and types) to dictionary key columns (and types).
                /// Call nested wrapper in order to cast dictionary keys. Then restore block.
                auto prev_arg_col = arg.column;
                auto prev_arg_type = arg.type;
                auto prev_res_type = res.type;

                auto tmp_rows_count = input_rows_count;

2153 2154
                if (to_low_cardinality)
                    res.type = to_low_cardinality->getDictionaryType();
2155

2156
                if (from_low_cardinality)
2157
                {
2158
                    auto * col_low_cardinality = typeid_cast<const ColumnLowCardinality *>(prev_arg_col.get());
2159 2160 2161 2162 2163

                    if (skip_not_null_check && col_low_cardinality->containsNull())
                        throw Exception{"Cannot convert NULL value to non-Nullable type",
                                        ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};

2164 2165
                    arg.column = col_low_cardinality->getDictionary().getNestedColumn();
                    arg.type = from_low_cardinality->getDictionaryType();
2166

2167 2168 2169
                    /// TODO: Make map with defaults conversion.
                    src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res.type));
                    if (src_converted_to_full_column)
2170
                        arg.column = arg.column->index(col_low_cardinality->getIndexes(), 0);
2171
                    else
2172
                        res_indexes = col_low_cardinality->getIndexesPtr();
2173

2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184
                    tmp_rows_count = arg.column->size();
                }

                /// Perform the requested conversion.
                wrapper(block, arguments, result, tmp_rows_count);

                arg.column = prev_arg_col;
                arg.type = prev_arg_type;
                res.type = prev_res_type;
            }

2185
            if (to_low_cardinality)
2186
            {
2187 2188
                auto res_column = to_low_cardinality->createColumn();
                auto * col_low_cardinality = typeid_cast<ColumnLowCardinality *>(res_column.get());
2189

2190
                if (from_low_cardinality && !src_converted_to_full_column)
2191 2192
                {
                    auto res_keys = std::move(res.column);
2193
                    col_low_cardinality->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes);
2194 2195
                }
                else
2196
                    col_low_cardinality->insertRangeFromFullColumn(*res.column, 0, res.column->size());
2197 2198 2199

                res.column = std::move(res_column);
            }
2200
            else if (!src_converted_to_full_column)
2201
                res.column = res.column->index(*res_indexes, 0);
2202 2203 2204
        };
    }

2205
    WrapperType prepareRemoveNullable(const DataTypePtr & from_type, const DataTypePtr & to_type, bool skip_not_null_check) const
2206 2207 2208
    {
        /// Determine whether pre-processing and/or post-processing must take place during conversion.

N
Nikolai Kochetov 已提交
2209 2210
        bool source_is_nullable = from_type->isNullable();
        bool result_is_nullable = to_type->isNullable();
2211

2212
        auto wrapper = prepareImpl(removeNullable(from_type), removeNullable(to_type), result_is_nullable);
2213

2214
        if (result_is_nullable)
2215
        {
2216 2217
            return [wrapper, source_is_nullable]
                (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
2218 2219
            {
                /// Create a temporary block on which to perform the operation.
2220
                auto & res = block.getByPosition(result);
2221 2222 2223 2224 2225
                const auto & ret_type = res.type;
                const auto & nullable_type = static_cast<const DataTypeNullable &>(*ret_type);
                const auto & nested_type = nullable_type.getNestedType();

                Block tmp_block;
2226
                if (source_is_nullable)
2227 2228 2229 2230 2231 2232 2233 2234
                    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.
T
Tsarkova Anastasia 已提交
2235
                wrapper(tmp_block, arguments, tmp_res_index, input_rows_count);
2236

2237
                const auto & tmp_res = tmp_block.getByPosition(tmp_res_index);
2238

2239 2240 2241 2242 2243
                /// May happen in fuzzy tests. For debug purpose.
                if (!tmp_res.column)
                    throw Exception("Couldn't convert " + block.getByPosition(arguments[0]).type->getName() + " to "
                                    + nested_type->getName() + " in " + " prepareRemoveNullable wrapper.", ErrorCodes::LOGICAL_ERROR);

2244
                res.column = wrapInNullable(tmp_res.column, Block({block.getByPosition(arguments[0]), tmp_res}), {0}, 1, input_rows_count);
2245 2246
            };
        }
2247
        else if (source_is_nullable)
2248 2249 2250
        {
            /// Conversion from Nullable to non-Nullable.

2251
            return [wrapper, skip_not_null_check] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
2252 2253 2254 2255
            {
                Block tmp_block = createBlockWithNestedColumns(block, arguments, result);

                /// Check that all values are not-NULL.
2256 2257 2258 2259 2260
                /// Check can be skipped in case if LowCardinality dictionary is transformed.
                /// In that case, correctness will be checked beforehand.
                if (!skip_not_null_check)
                {
                    const auto & col = block.getByPosition(arguments[0]).column;
2261
                    const auto & nullable_col = assert_cast<const ColumnNullable &>(*col);
2262
                    const auto & null_map = nullable_col.getNullMapData();
2263

2264 2265 2266 2267
                    if (!memoryIsZero(null_map.data(), null_map.size()))
                        throw Exception{"Cannot convert NULL value to non-Nullable type",
                                        ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
                }
2268

T
Tsarkova Anastasia 已提交
2269
                wrapper(tmp_block, arguments, result, input_rows_count);
2270 2271 2272
                block.getByPosition(result).column = tmp_block.getByPosition(result).column;
            };
        }
2273 2274 2275 2276
        else
            return wrapper;
    }

C
chertus 已提交
2277 2278
    /// 'from_type' and 'to_type' are nested types in case of Nullable.
    /// 'requested_result_is_nullable' is true if CAST to Nullable type is requested.
2279
    WrapperType prepareImpl(const DataTypePtr & from_type, const DataTypePtr & to_type, bool requested_result_is_nullable) const
2280 2281 2282
    {
        if (from_type->equals(*to_type))
            return createIdentityWrapper(from_type);
2283
        else if (WhichDataType(from_type).isNothing())
2284
            return createNothingWrapper(to_type.get());
C
chertus 已提交
2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319

        WrapperType ret;

        auto make_default_wrapper = [&](const auto & types) -> bool
        {
            using Types = std::decay_t<decltype(types)>;
            using ToDataType = typename Types::LeftType;

            if constexpr (
                std::is_same_v<ToDataType, DataTypeUInt8> ||
                std::is_same_v<ToDataType, DataTypeUInt16> ||
                std::is_same_v<ToDataType, DataTypeUInt32> ||
                std::is_same_v<ToDataType, DataTypeUInt64> ||
                std::is_same_v<ToDataType, DataTypeInt8> ||
                std::is_same_v<ToDataType, DataTypeInt16> ||
                std::is_same_v<ToDataType, DataTypeInt32> ||
                std::is_same_v<ToDataType, DataTypeInt64> ||
                std::is_same_v<ToDataType, DataTypeFloat32> ||
                std::is_same_v<ToDataType, DataTypeFloat64> ||
                std::is_same_v<ToDataType, DataTypeDate> ||
                std::is_same_v<ToDataType, DataTypeDateTime>)
            {
                ret = createWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
                return true;
            }
            if constexpr (
                std::is_same_v<ToDataType, DataTypeEnum8> ||
                std::is_same_v<ToDataType, DataTypeEnum16>)
            {
                ret = createEnumWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
                return true;
            }
            if constexpr (
                std::is_same_v<ToDataType, DataTypeDecimal<Decimal32>> ||
                std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> ||
2320
                std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>> ||
2321
                std::is_same_v<ToDataType, DataTypeDateTime64>)
C
chertus 已提交
2322 2323 2324 2325
            {
                ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
                return true;
            }
2326 2327 2328 2329 2330 2331 2332 2333
            if constexpr (std::is_same_v<ToDataType, DataTypeUUID>)
            {
                if (isStringOrFixedString(from_type))
                {
                    ret = createUUIDWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
                    return true;
                }
            }
C
chertus 已提交
2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351 2352

            return false;
        };

        if (callOnIndexAndDataType<void>(to_type->getTypeId(), make_default_wrapper))
            return ret;

        switch (to_type->getTypeId())
        {
            case TypeIndex::String:
                return createStringWrapper(from_type);
            case TypeIndex::FixedString:
                return createFixedStringWrapper(from_type, checkAndGetDataType<DataTypeFixedString>(to_type.get())->getN());

            case TypeIndex::Array:
                return createArrayWrapper(from_type, checkAndGetDataType<DataTypeArray>(to_type.get()));
            case TypeIndex::Tuple:
                return createTupleWrapper(from_type, checkAndGetDataType<DataTypeTuple>(to_type.get()));

2353 2354
            case TypeIndex::AggregateFunction:
                return createAggregateFunctionWrapper(from_type, checkAndGetDataType<DataTypeAggregateFunction>(to_type.get()));
C
chertus 已提交
2355 2356 2357
            default:
                break;
        }
2358

C
chertus 已提交
2359 2360
        throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
            ErrorCodes::CANNOT_CONVERT_TYPE};
2361
    }
N
Nikolai Kochetov 已提交
2362 2363
};

2364
class CastOverloadResolver : public IFunctionOverloadResolverImpl
N
Nikolai Kochetov 已提交
2365 2366 2367 2368 2369
{
public:
    using MonotonicityForRange = FunctionCast::MonotonicityForRange;

    static constexpr auto name = "CAST";
2370 2371
    static FunctionOverloadResolverImplPtr create(const Context &) { return createImpl(); }
    static FunctionOverloadResolverImplPtr createImpl() { return std::make_unique<CastOverloadResolver>(); }
N
Nikolai Kochetov 已提交
2372

2373
    CastOverloadResolver() {}
N
Nikolai Kochetov 已提交
2374

A
Alexey Milovidov 已提交
2375
    String getName() const override { return name; }
N
Nikolai Kochetov 已提交
2376 2377 2378

    size_t getNumberOfArguments() const override { return 2; }

A
Alexander Tokmakov 已提交
2379
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
2380

N
Nikolai Kochetov 已提交
2381 2382
protected:

N
Nikolai Kochetov 已提交
2383
    FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override
N
Nikolai Kochetov 已提交
2384 2385 2386 2387 2388 2389
    {
        DataTypes data_types(arguments.size());

        for (size_t i = 0; i < arguments.size(); ++i)
            data_types[i] = arguments[i].type;

2390
        auto monotonicity = getMonotonicityInformation(arguments.front().type, return_type.get());
2391
        return std::make_unique<FunctionCast>(name, std::move(monotonicity), data_types, return_type);
N
Nikolai Kochetov 已提交
2392 2393
    }

N
Nikolai Kochetov 已提交
2394
    DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments) const override
N
Nikolai Kochetov 已提交
2395 2396 2397
    {
        const auto type_col = checkAndGetColumnConst<ColumnString>(arguments.back().column.get());
        if (!type_col)
C
chertus 已提交
2398 2399
            throw Exception("Second argument to " + getName() + " must be a constant string describing type",
                            ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
N
Nikolai Kochetov 已提交
2400 2401 2402 2403 2404

        return DataTypeFactory::instance().get(type_col->getValue<String>());
    }

    bool useDefaultImplementationForNulls() const override { return false; }
2405
    bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
N
Nikolai Kochetov 已提交
2406 2407

private:
2408 2409
    template <typename DataType>
    static auto monotonicityForType(const DataType * const)
2410 2411 2412 2413
    {
        return FunctionTo<DataType>::Type::Monotonic::get;
    }

N
Nikolai Kochetov 已提交
2414
    MonotonicityForRange getMonotonicityInformation(const DataTypePtr & from_type, const IDataType * to_type) const
2415
    {
2416
        if (const auto type = checkAndGetDataType<DataTypeUInt8>(to_type))
N
Nikolai Kochetov 已提交
2417
            return monotonicityForType(type);
2418
        if (const auto type = checkAndGetDataType<DataTypeUInt16>(to_type))
N
Nikolai Kochetov 已提交
2419
            return monotonicityForType(type);
2420
        if (const auto type = checkAndGetDataType<DataTypeUInt32>(to_type))
N
Nikolai Kochetov 已提交
2421
            return monotonicityForType(type);
2422
        if (const auto type = checkAndGetDataType<DataTypeUInt64>(to_type))
N
Nikolai Kochetov 已提交
2423
            return monotonicityForType(type);
2424
        if (const auto type = checkAndGetDataType<DataTypeInt8>(to_type))
N
Nikolai Kochetov 已提交
2425
            return monotonicityForType(type);
2426
        if (const auto type = checkAndGetDataType<DataTypeInt16>(to_type))
N
Nikolai Kochetov 已提交
2427
            return monotonicityForType(type);
2428
        if (const auto type = checkAndGetDataType<DataTypeInt32>(to_type))
N
Nikolai Kochetov 已提交
2429
            return monotonicityForType(type);
2430
        if (const auto type = checkAndGetDataType<DataTypeInt64>(to_type))
N
Nikolai Kochetov 已提交
2431
            return monotonicityForType(type);
2432
        if (const auto type = checkAndGetDataType<DataTypeFloat32>(to_type))
N
Nikolai Kochetov 已提交
2433
            return monotonicityForType(type);
2434
        if (const auto type = checkAndGetDataType<DataTypeFloat64>(to_type))
N
Nikolai Kochetov 已提交
2435
            return monotonicityForType(type);
2436
        if (const auto type = checkAndGetDataType<DataTypeDate>(to_type))
N
Nikolai Kochetov 已提交
2437
            return monotonicityForType(type);
2438
        if (const auto type = checkAndGetDataType<DataTypeDateTime>(to_type))
N
Nikolai Kochetov 已提交
2439
            return monotonicityForType(type);
2440
        if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
N
Nikolai Kochetov 已提交
2441
            return monotonicityForType(type);
2442
        if (isEnum(from_type))
2443
        {
2444
            if (const auto type = checkAndGetDataType<DataTypeEnum8>(to_type))
N
Nikolai Kochetov 已提交
2445
                return monotonicityForType(type);
2446
            if (const auto type = checkAndGetDataType<DataTypeEnum16>(to_type))
N
Nikolai Kochetov 已提交
2447
                return monotonicityForType(type);
2448 2449
        }
        /// other types like Null, FixedString, Array and Tuple have no monotonicity defined
N
Nikolai Kochetov 已提交
2450
        return {};
2451
    }
2452 2453
};

A
Alexey Milovidov 已提交
2454
}