mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-09 17:14:47 +00:00
Extend with origin argument, pt. I
This commit is contained in:
parent
cd94eda704
commit
aef1ea3df7
@ -321,6 +321,11 @@ public:
|
||||
size_t getNumberOfArguments() const override { return 0; }
|
||||
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; }
|
||||
|
||||
bool useDefaultImplementationForConstants() const override { return true; }
|
||||
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
||||
|
||||
bool hasInformationAboutMonotonicity() const override { return true; }
|
||||
Monotonicity getMonotonicityForRange(const IDataType &, const Field &, const Field &) const override { return { .is_monotonic = true, .is_always_monotonic = true }; }
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
@ -329,7 +334,7 @@ public:
|
||||
{
|
||||
const DataTypePtr & type_arg1 = arguments[0].type;
|
||||
if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 1st argument of function {}. "
|
||||
"Should be a date or a date with time", type_arg1->getName(), getName());
|
||||
value_is_date = isDate(type_arg1);
|
||||
};
|
||||
@ -347,7 +352,7 @@ public:
|
||||
const DataTypePtr & type_arg2 = arguments[1].type;
|
||||
interval_type = checkAndGetDataType<DataTypeInterval>(type_arg2.get());
|
||||
if (!interval_type)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. "
|
||||
"Should be an interval of time", type_arg2->getName(), getName());
|
||||
switch (interval_type->getKind()) // NOLINT(bugprone-switch-missing-default-case)
|
||||
{
|
||||
@ -371,18 +376,48 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
enum class ThirdArgument
|
||||
{
|
||||
IsTimezone,
|
||||
IsOrigin
|
||||
};
|
||||
ThirdArgument third_argument; /// valid only if 3rd argument is given
|
||||
auto check_third_argument = [&]
|
||||
{
|
||||
const DataTypePtr & type_arg3 = arguments[2].type;
|
||||
if (!isString(type_arg3))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}. "
|
||||
"This argument is optional and must be a constant string with timezone name",
|
||||
if (isString(type_arg3))
|
||||
{
|
||||
third_argument = ThirdArgument::IsTimezone;
|
||||
if (value_is_date && result_type == ResultType::Date)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64",
|
||||
getName(), interval_type->getKind().toString());
|
||||
}
|
||||
else if (isDateOrDate32OrDateTimeOrDateTime64(type_arg3))
|
||||
third_argument = ThirdArgument::IsOrigin;
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. "
|
||||
"This argument is optional and must be a constant String with timezone name or a Date/Date32/DateTime/DateTime64 with a constant origin",
|
||||
type_arg3->getName(), getName());
|
||||
|
||||
};
|
||||
|
||||
auto check_fourth_argument = [&]
|
||||
{
|
||||
if (third_argument != ThirdArgument::IsOrigin) /// sanity check
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 3rd argument of function {}. "
|
||||
"The third argument must a Date/Date32/DateTime/DateTime64 with a constant origin",
|
||||
arguments[2].type->getName(), getName());
|
||||
|
||||
const DataTypePtr & type_arg4 = arguments[3].type;
|
||||
if (!isString(type_arg4))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 4th argument of function {}. "
|
||||
"This argument is optional and must be a constant String with timezone name",
|
||||
type_arg4->getName(), getName());
|
||||
if (value_is_date && result_type == ResultType::Date)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument "
|
||||
"has the type DateTime or DateTime64",
|
||||
getName(), interval_type->getKind().toString());
|
||||
"The timezone argument of function {} with interval type {} is allowed only when the 1st argument has the type DateTime or DateTime64",
|
||||
getName(), interval_type->getKind().toString());
|
||||
};
|
||||
|
||||
if (arguments.size() == 2)
|
||||
@ -396,10 +431,17 @@ public:
|
||||
check_second_argument();
|
||||
check_third_argument();
|
||||
}
|
||||
else if (arguments.size() == 4)
|
||||
{
|
||||
check_first_argument();
|
||||
check_second_argument();
|
||||
check_third_argument();
|
||||
check_fourth_argument();
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2 or 3",
|
||||
"Number of arguments for function {} doesn't match: passed {}, should be 2, 3 or 4",
|
||||
getName(), arguments.size());
|
||||
}
|
||||
|
||||
@ -408,7 +450,10 @@ public:
|
||||
case ResultType::Date:
|
||||
return std::make_shared<DataTypeDate>();
|
||||
case ResultType::DateTime:
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
|
||||
{
|
||||
const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3;
|
||||
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false));
|
||||
}
|
||||
case ResultType::DateTime64:
|
||||
{
|
||||
UInt32 scale = 0;
|
||||
@ -419,7 +464,8 @@ public:
|
||||
else if (interval_type->getKind() == IntervalKind::Millisecond)
|
||||
scale = 3;
|
||||
|
||||
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false));
|
||||
const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && third_argument == ThirdArgument::IsTimezone)) ? 2 : 3;
|
||||
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, time_zone_arg_num, 0, false));
|
||||
}
|
||||
}
|
||||
|
||||
@ -433,8 +479,16 @@ public:
|
||||
{
|
||||
const auto & time_column = arguments[0];
|
||||
const auto & interval_column = arguments[1];
|
||||
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0);
|
||||
auto result_column = dispatchForColumns(time_column, interval_column, result_type, time_zone);
|
||||
|
||||
ColumnWithTypeAndName origin_column;
|
||||
const bool has_origin_arg = (arguments.size() == 3 && isDateOrDate32OrDateTimeOrDateTime64(arguments[2].type)) || arguments.size() == 4;
|
||||
if (has_origin_arg)
|
||||
origin_column = arguments[2];
|
||||
|
||||
const size_t time_zone_arg_num = (arguments.size() == 2 || (arguments.size() == 3 && isString(arguments[2].type))) ? 2 : 3;
|
||||
const auto & time_zone = extractTimeZoneFromFunctionArguments(arguments, time_zone_arg_num, 0);
|
||||
|
||||
auto result_column = dispatchForTimeColumn(time_column, interval_column, origin_column, result_type, time_zone);
|
||||
return result_column;
|
||||
}
|
||||
|
||||
@ -449,8 +503,8 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
ColumnPtr dispatchForColumns(
|
||||
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
|
||||
ColumnPtr dispatchForTimeColumn(
|
||||
const ColumnWithTypeAndName & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone) const
|
||||
{
|
||||
const auto & time_column_type = *time_column.type.get();
|
||||
const auto & time_column_col = *time_column.column.get();
|
||||
@ -461,25 +515,25 @@ private:
|
||||
auto scale = assert_cast<const DataTypeDateTime64 &>(time_column_type).getScale();
|
||||
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone, scale);
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime64 &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone, scale);
|
||||
}
|
||||
else if (isDateTime(time_column_type))
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnDateTime>(time_column_col);
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone);
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDateTime &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
|
||||
}
|
||||
else if (isDate(time_column_type))
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnDate>(time_column_col);
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone);
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
|
||||
}
|
||||
else if (isDate32(time_column_type))
|
||||
{
|
||||
const auto * time_column_vec = checkAndGetColumn<ColumnDate32>(time_column_col);
|
||||
if (time_column_vec)
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32 &>(time_column_type), *time_column_vec, interval_column, result_type, time_zone);
|
||||
return dispatchForIntervalColumn(assert_cast<const DataTypeDate32 &>(time_column_type), *time_column_vec, interval_column, origin_column, result_type, time_zone);
|
||||
}
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. "
|
||||
"Must contain dates or dates with time", getName());
|
||||
@ -487,7 +541,7 @@ private:
|
||||
|
||||
template <typename ColumnType, typename FromDataType>
|
||||
ColumnPtr dispatchForIntervalColumn(
|
||||
const FromDataType & from, const ColumnType & time_column, const ColumnWithTypeAndName & interval_column,
|
||||
const TimeDataType & time_data_type, const TimeColumnType & time_column, const ColumnWithTypeAndName & interval_column, const ColumnWithTypeAndName & origin_column,
|
||||
const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale = 1) const
|
||||
{
|
||||
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
|
||||
@ -505,34 +559,34 @@ private:
|
||||
switch (interval_type->getKind())
|
||||
{
|
||||
case IntervalKind::Nanosecond:
|
||||
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Nanosecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Microsecond:
|
||||
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Microsecond>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Microsecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Millisecond:
|
||||
return execute<FromDataType, DataTypeDateTime64, IntervalKind::Millisecond>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime64, IntervalKind::Millisecond>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Second:
|
||||
return execute<FromDataType, DataTypeDateTime, IntervalKind::Second>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Second>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Minute:
|
||||
return execute<FromDataType, DataTypeDateTime, IntervalKind::Minute>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Minute>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Hour:
|
||||
return execute<FromDataType, DataTypeDateTime, IntervalKind::Hour>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Hour>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Day:
|
||||
return execute<FromDataType, DataTypeDateTime, IntervalKind::Day>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDateTime, IntervalKind::Day>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Week:
|
||||
return execute<FromDataType, DataTypeDate, IntervalKind::Week>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDate, IntervalKind::Week>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Month:
|
||||
return execute<FromDataType, DataTypeDate, IntervalKind::Month>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDate, IntervalKind::Month>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Quarter:
|
||||
return execute<FromDataType, DataTypeDate, IntervalKind::Quarter>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDate, IntervalKind::Quarter>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
case IntervalKind::Year:
|
||||
return execute<FromDataType, DataTypeDate, IntervalKind::Year>(from, time_column, num_units, result_type, time_zone, scale);
|
||||
return execute<TimeDataType, DataTypeDate, IntervalKind::Year>(time_data_type, time_column, num_units, origin_column, result_type, time_zone, scale);
|
||||
}
|
||||
|
||||
std::unreachable();
|
||||
}
|
||||
|
||||
template <typename FromDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
|
||||
ColumnPtr execute(const FromDataType &, const ColumnType & time_column_type, Int64 num_units, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
|
||||
template <typename TimeDataType, typename ToDataType, IntervalKind::Kind unit, typename ColumnType>
|
||||
ColumnPtr execute(const TimeDataType &, const ColumnType & time_column_type, Int64 num_units, [[maybe_unused]] const ColumnWithTypeAndName & origin_column, const DataTypePtr & result_type, const DateLUTImpl & time_zone, const UInt16 scale) const
|
||||
{
|
||||
using ToColumnType = typename ToDataType::ColumnType;
|
||||
using ToFieldType = typename ToDataType::FieldType;
|
||||
@ -547,6 +601,14 @@ private:
|
||||
|
||||
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
|
||||
|
||||
/// TODO: This part is missing. origin_column is either {} (<-- to check, you could do `origin_column.column == nullptr`) or not {}
|
||||
/// In the former case, we can execute below existing code.
|
||||
/// In the latter case, we need to read the actual origin value. As per `getArgumentsThatAreAlwaysConstant()` (see above), we
|
||||
/// can be sure that origin_column is a `ColumnConst`. The second assumption we can reasonable make is that it has the same
|
||||
/// type (Date/Date32/DateTime/DateTime64) as the time column (1st argument). Since the method we are in is already
|
||||
/// templatized on the data type of the time column, we can use `checkAndGetColumnConst<ColumnType>(...)` to cast
|
||||
/// `origin_column.column` to a const column and then read the (const) value from it, and proceed with the calculations.
|
||||
|
||||
for (size_t i = 0; i != size; ++i)
|
||||
result_data[i] = static_cast<ToFieldType>(
|
||||
Transform<unit>::execute(time_data[i], num_units, time_zone, scale_multiplier));
|
||||
|
Loading…
Reference in New Issue
Block a user