ClickHouse

Форк
0
/
date_trunc.cpp 
165 строк · 6.7 Кб
1
#include <Columns/ColumnConst.h>
2
#include <Columns/ColumnString.h>
3
#include <Columns/ColumnsNumber.h>
4
#include <DataTypes/DataTypeDate.h>
5
#include <DataTypes/DataTypeDateTime.h>
6
#include <DataTypes/DataTypeInterval.h>
7
#include <Formats/FormatSettings.h>
8
#include <Functions/DateTimeTransforms.h>
9
#include <Functions/FunctionFactory.h>
10

11

12
namespace DB
13
{
14
namespace ErrorCodes
15
{
16
    extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
17
    extern const int ILLEGAL_TYPE_OF_ARGUMENT;
18
    extern const int BAD_ARGUMENTS;
19
}
20

21
namespace
22
{
23

24
class FunctionDateTrunc : public IFunction
25
{
26
public:
27
    static constexpr auto name = "dateTrunc";
28

29
    explicit FunctionDateTrunc(ContextPtr context_) : context(context_) {}
30

31
    static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionDateTrunc>(context); }
32

33
    String getName() const override { return name; }
34

35
    bool isVariadic() const override { return true; }
36
    bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
37
    size_t getNumberOfArguments() const override { return 0; }
38

39
    DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
40
    {
41
        /// The first argument is a constant string with the name of datepart.
42

43
        auto result_type_is_date = false;
44
        String datepart_param;
45
        auto check_first_argument = [&] {
46
            const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
47
            if (!datepart_column)
48
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be constant string: "
49
                    "name of datepart", getName());
50

51
            datepart_param = Poco::toLower(datepart_column->getValue<String>());
52
            if (datepart_param.empty())
53
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument (name of datepart) for function {} cannot be empty",
54
                    getName());
55

56
            if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
57
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
58

59
            if (datepart_kind == IntervalKind::Kind::Nanosecond || datepart_kind == IntervalKind::Kind::Microsecond
60
                || datepart_kind == IntervalKind::Kind::Millisecond)
61
                throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't support {}", getName(), datepart_param);
62

63
            result_type_is_date = (datepart_kind == IntervalKind::Kind::Year)
64
                || (datepart_kind == IntervalKind::Kind::Quarter) || (datepart_kind == IntervalKind::Kind::Month)
65
                || (datepart_kind == IntervalKind::Kind::Week);
66
        };
67

68
        bool second_argument_is_date = false;
69
        auto check_second_argument = [&] {
70
            if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type))
71
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. "
72
                    "Should be a date or a date with time", arguments[1].type->getName(), getName());
73

74
            second_argument_is_date = isDate(arguments[1].type);
75

76
            if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour)
77
                || (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second)))
78
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName());
79
        };
80

81
        auto check_timezone_argument = [&] {
82
            if (!WhichDataType(arguments[2].type).isString())
83
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
84
                    "This argument is optional and must be a constant string with timezone name",
85
                    arguments[2].type->getName(), getName());
86

87
            if (second_argument_is_date && result_type_is_date)
88
                throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
89
                                "The timezone argument of function {} with datepart '{}' "
90
                                "is allowed only when the 2nd argument has the type DateTime",
91
                                getName(), datepart_param);
92
        };
93

94
        if (arguments.size() == 2)
95
        {
96
            check_first_argument();
97
            check_second_argument();
98
        }
99
        else if (arguments.size() == 3)
100
        {
101
            check_first_argument();
102
            check_second_argument();
103
            check_timezone_argument();
104
        }
105
        else
106
        {
107
            throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
108
                "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
109
                getName(), arguments.size());
110
        }
111

112
        if (result_type_is_date)
113
            return std::make_shared<DataTypeDate>();
114
        else
115
            return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
116
    }
117

118
    bool useDefaultImplementationForConstants() const override { return true; }
119
    ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0, 2}; }
120

121
    ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
122
    {
123
        ColumnsWithTypeAndName temp_columns(arguments.size());
124
        temp_columns[0] = arguments[1];
125

126
        const UInt16 interval_value = 1;
127
        const ColumnPtr interval_column = ColumnConst::create(ColumnInt64::create(1, interval_value), input_rows_count);
128
        temp_columns[1] = {interval_column, std::make_shared<DataTypeInterval>(datepart_kind), ""};
129

130
        auto to_start_of_interval = FunctionFactory::instance().get("toStartOfInterval", context);
131

132
        if (arguments.size() == 2)
133
            return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
134

135
        temp_columns[2] = arguments[2];
136
        return to_start_of_interval->build(temp_columns)->execute(temp_columns, result_type, input_rows_count);
137
    }
138

139
    bool hasInformationAboutMonotonicity() const override
140
    {
141
        return true;
142
    }
143

144
    Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override
145
    {
146
        return { .is_monotonic = true, .is_always_monotonic = true };
147
    }
148

149
private:
150
    ContextPtr context;
151
    mutable IntervalKind::Kind datepart_kind = IntervalKind::Kind::Second;
152
};
153

154
}
155

156

157
REGISTER_FUNCTION(DateTrunc)
158
{
159
    factory.registerFunction<FunctionDateTrunc>();
160

161
    /// Compatibility alias.
162
    factory.registerAlias("DATE_TRUNC", "dateTrunc", FunctionFactory::CaseInsensitive);
163
}
164

165
}
166

Использование cookies

Мы используем файлы cookie в соответствии с Политикой конфиденциальности и Политикой использования cookies.

Нажимая кнопку «Принимаю», Вы даете АО «СберТех» согласие на обработку Ваших персональных данных в целях совершенствования нашего веб-сайта и Сервиса GitVerse, а также повышения удобства их использования.

Запретить использование cookies Вы можете самостоятельно в настройках Вашего браузера.