Merge pull request #30927 from CurtizJ/with-fill-interval

Support `INTERVAL` type in `STEP` clause for `WITH FILL` modifier
This commit is contained in:
Kruglov Pavel 2021-11-03 11:23:21 +03:00 committed by GitHub
commit 2f950ee138
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 337 additions and 47 deletions

View File

@ -25,7 +25,6 @@ Int32 IntervalKind::toAvgSeconds() const
__builtin_unreachable();
}
IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds)
{
if (num_seconds)

View File

@ -58,4 +58,15 @@ struct IntervalKind
/// For example, `IntervalKind::tryParseString('second', result)` returns `result` equals `IntervalKind::Kind::Second`.
static bool tryParseString(const std::string & kind, IntervalKind::Kind & result);
};
#define FOR_EACH_INTERVAL_KIND(M) \
M(Second) \
M(Minute) \
M(Hour) \
M(Day) \
M(Week) \
M(Month) \
M(Quarter) \
M(Year)
}

View File

@ -6,6 +6,7 @@
#include <string>
#include <Core/Field.h>
#include <Core/SettingsEnums.h>
#include <Common/IntervalKind.h>
class Collator;
@ -27,7 +28,11 @@ struct FillColumnDescription
/// Range [FROM, TO) respects sorting direction
Field fill_from; /// Fill value >= FILL_FROM
Field fill_to; /// Fill value + STEP < FILL_TO
Field fill_step; /// Default = 1 or -1 according to direction
Field fill_step; /// Default = +1 or -1 according to direction
std::optional<IntervalKind> step_kind;
using StepFunction = std::function<void(Field &)>;
StepFunction step_func;
};
/// Description of the sorting rule by one column.

View File

@ -64,7 +64,7 @@ bool FillingRow::next(const FillingRow & to_row)
continue;
auto next_value = row[i];
applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value);
getFillDescription(i).step_func(next_value);
if (less(next_value, getFillDescription(i).fill_to, getDirection(i)))
{
row[i] = next_value;
@ -74,7 +74,7 @@ bool FillingRow::next(const FillingRow & to_row)
}
auto next_value = row[pos];
applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value);
getFillDescription(pos).step_func(next_value);
if (less(to_row[pos], next_value, getDirection(pos)))
return false;

View File

@ -1,4 +1,5 @@
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeInterval.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -715,7 +716,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
auto [field, type] = evaluateConstantExpression(node, context);
if (!isColumnedAsNumber(type))
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
@ -723,6 +724,19 @@ static Field getWithFillFieldValue(const ASTPtr & node, ContextPtr context)
return field;
}
static std::pair<Field, std::optional<IntervalKind>> getWithFillStep(const ASTPtr & node, ContextPtr context)
{
auto [field, type] = evaluateConstantExpression(node, context);
if (const auto * type_interval = typeid_cast<const DataTypeInterval *>(type.get()))
return std::make_pair(std::move(field), type_interval->getKind());
if (isColumnedAsNumber(type))
return std::make_pair(std::move(field), std::nullopt);
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, ContextPtr context)
{
FillColumnDescription descr;
@ -730,8 +744,9 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement & or
descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context);
if (order_by_elem.fill_to)
descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context);
if (order_by_elem.fill_step)
descr.fill_step = getWithFillFieldValue(order_by_elem.fill_step, context);
std::tie(descr.fill_step, descr.step_kind) = getWithFillStep(order_by_elem.fill_step, context);
else
descr.fill_step = order_by_elem.direction;

View File

@ -5,6 +5,9 @@
#include <DataTypes/IDataType.h>
#include <Core/Types.h>
#include <DataTypes/DataTypesDecimal.h>
#include <Functions/FunctionDateOrDateTimeAddInterval.h>
#include <Common/FieldVisitorSum.h>
#include <Common/FieldVisitorToString.h>
namespace DB
@ -29,6 +32,113 @@ Block FillingTransform::transformHeader(Block header, const SortDescription & so
return header;
}
template <typename T>
static FillColumnDescription::StepFunction getStepFunction(
IntervalKind kind, Int64 step, const DateLUTImpl & date_lut)
{
switch (kind)
{
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
return [step, &date_lut](Field & field) { field = Add##NAME##sImpl::execute(get<T>(field), step, date_lut); };
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
}
__builtin_unreachable();
}
static bool tryConvertFields(FillColumnDescription & descr, const DataTypePtr & type)
{
auto max_type = Field::Types::Null;
WhichDataType which(type);
DataTypePtr to_type;
/// TODO Wrong results for big integers.
if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime())
{
max_type = Field::Types::Int64;
to_type = std::make_shared<DataTypeInt64>();
}
else if (which.isDateTime64())
{
max_type = Field::Types::Decimal64;
const auto & date_type = static_cast<const DataTypeDateTime64 &>(*type);
size_t precision = date_type.getPrecision();
size_t scale = date_type.getScale();
to_type = std::make_shared<DataTypeDecimal<Decimal64>>(precision, scale);
}
else if (which.isFloat())
{
max_type = Field::Types::Float64;
to_type = std::make_shared<DataTypeFloat64>();
}
else
return false;
if (descr.fill_from.getType() > max_type
|| descr.fill_to.getType() > max_type
|| descr.fill_step.getType() > max_type)
return false;
descr.fill_from = convertFieldToType(descr.fill_from, *to_type);
descr.fill_to = convertFieldToType(descr.fill_to, *to_type);
descr.fill_step = convertFieldToType(descr.fill_step, *to_type);
if (descr.step_kind)
{
if (which.isDate() || which.isDate32())
{
Int64 avg_seconds = get<Int64>(descr.fill_step) * descr.step_kind->toAvgSeconds();
if (avg_seconds < 86400)
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"Value of step is to low ({} seconds). Must be >= 1 day", avg_seconds);
}
if (which.isDate())
descr.step_func = getStepFunction<UInt16>(*descr.step_kind, get<Int64>(descr.fill_step), DateLUT::instance());
else if (which.isDate32())
descr.step_func = getStepFunction<Int32>(*descr.step_kind, get<Int64>(descr.fill_step), DateLUT::instance());
else if (const auto * date_time = checkAndGetDataType<DataTypeDateTime>(type.get()))
descr.step_func = getStepFunction<UInt32>(*descr.step_kind, get<Int64>(descr.fill_step), date_time->getTimeZone());
else if (const auto * date_time64 = checkAndGetDataType<DataTypeDateTime64>(type.get()))
{
const auto & step_dec = get<const DecimalField<Decimal64> &>(descr.fill_step);
Int64 step = DecimalUtils::convertTo<Int64>(step_dec.getValue(), step_dec.getScale());
switch (*descr.step_kind)
{
#define DECLARE_CASE(NAME) \
case IntervalKind::NAME: \
descr.step_func = [step, &time_zone = date_time64->getTimeZone()](Field & field) \
{ \
auto field_decimal = get<DecimalField<DateTime64>>(field); \
auto components = DecimalUtils::splitWithScaleMultiplier(field_decimal.getValue(), field_decimal.getScaleMultiplier()); \
auto res = Add##NAME##sImpl::execute(components, step, time_zone); \
auto res_decimal = decimalFromComponentsWithMultiplier<DateTime64>(res, field_decimal.getScaleMultiplier()); \
field = DecimalField(res_decimal, field_decimal.getScale()); \
}; \
break;
FOR_EACH_INTERVAL_KIND(DECLARE_CASE)
#undef DECLARE_CASE
}
}
else
throw Exception(ErrorCodes::INVALID_WITH_FILL_EXPRESSION,
"STEP of Interval type can be used only with Date/DateTime types, but got {}", type->getName());
}
else
{
descr.step_func = [step = descr.fill_step](Field & field)
{
applyVisitor(FieldVisitorSum(step), field);
};
}
return true;
}
FillingTransform::FillingTransform(
const Block & header_, const SortDescription & sort_description_, bool on_totals_)
: ISimpleTransform(header_, transformHeader(header_, sort_description_), true)
@ -40,46 +150,6 @@ FillingTransform::FillingTransform(
if (on_totals)
return;
auto try_convert_fields = [](auto & descr, const auto & type)
{
auto max_type = Field::Types::Null;
WhichDataType which(type);
DataTypePtr to_type;
/// TODO Wrong results for big integers.
if (isInteger(type) || which.isDate() || which.isDate32() || which.isDateTime())
{
max_type = Field::Types::Int64;
to_type = std::make_shared<DataTypeInt64>();
}
else if (which.isDateTime64())
{
max_type = Field::Types::Decimal64;
const auto & date_type = static_cast<const DataTypeDateTime64 &>(*type);
size_t precision = date_type.getPrecision();
size_t scale = date_type.getScale();
to_type = std::make_shared<DataTypeDecimal<Decimal64>>(precision, scale);
}
else if (which.isFloat())
{
max_type = Field::Types::Float64;
to_type = std::make_shared<DataTypeFloat64>();
}
else
return false;
if (descr.fill_from.getType() > max_type
|| descr.fill_to.getType() > max_type
|| descr.fill_step.getType() > max_type)
return false;
descr.fill_from = convertFieldToType(descr.fill_from, *to_type);
descr.fill_to = convertFieldToType(descr.fill_to, *to_type);
descr.fill_step = convertFieldToType(descr.fill_step, *to_type);
return true;
};
std::vector<bool> is_fill_column(header_.columns());
for (size_t i = 0, size = sort_description.size(); i < size; ++i)
{
@ -90,7 +160,7 @@ FillingTransform::FillingTransform(
auto & descr = filling_row.getFillDescription(i);
const auto & type = header_.getByPosition(block_position).type;
if (!try_convert_fields(descr, type))
if (!tryConvertFields(descr, type))
throw Exception("Incompatible types of WITH FILL expression values with column type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);

View File

@ -0,0 +1,109 @@
1 DAY
2020-02-05 1
2020-02-06 0
2020-02-07 0
2020-02-08 0
2020-02-09 0
1 WEEK
2020-02-02 1
2020-02-09 0
2020-02-16 1
2020-02-23 0
2020-03-01 1
1 MONTH
2020-02-01 2
2020-03-01 1
2020-04-01 0
2020-05-01 0
2020-06-01 1
3 MONTH
2020-01-01 0
2020-02-01 2
2020-03-01 1
2020-04-01 0
2020-06-01 1
2020-07-01 0
2020-10-01 0
1 DAY
2020-02-05 1
2020-02-06 0
2020-02-07 0
2020-02-08 0
2020-02-09 0
1 WEEK
2020-02-02 1
2020-02-09 0
2020-02-16 1
2020-02-23 0
2020-03-01 1
1 MONTH
2020-02-01 2
2020-03-01 1
2020-04-01 0
2020-05-01 0
2020-06-01 1
3 MONTH
2020-01-01 0
2020-02-01 2
2020-03-01 1
2020-04-01 0
2020-06-01 1
2020-07-01 0
2020-10-01 0
15 MINUTE
2020-02-05 10:20:00 1
2020-02-05 10:35:00 0
2020-02-05 10:50:00 0
2020-02-05 11:05:00 0
2020-02-05 11:20:00 0
6 HOUR
2020-02-05 10:00:00 1
2020-02-05 16:00:00 0
2020-02-05 22:00:00 0
2020-02-06 04:00:00 0
2020-02-06 10:00:00 0
10 DAY
2020-02-05 00:00:00 1
2020-02-15 00:00:00 0
2020-02-25 00:00:00 0
2020-03-06 00:00:00 0
2020-03-08 00:00:00 1
15 MINUTE
2020-02-05 10:20:00.000 1
2020-02-05 10:35:00.000 0
2020-02-05 10:50:00.000 0
2020-02-05 11:05:00.000 0
2020-02-05 11:20:00.000 0
6 HOUR
2020-02-05 10:00:00 1
2020-02-05 16:00:00 0
2020-02-05 22:00:00 0
2020-02-06 04:00:00 0
2020-02-06 10:00:00 0
10 DAY
2020-02-05 00:00:00 1
2020-02-15 00:00:00 0
2020-02-25 00:00:00 0
2020-03-06 00:00:00 0
2020-03-08 00:00:00 1
1 MONTH
2020-01-01 1 0
2020-01-01 2 0
2020-01-01 3 0
2020-01-01 4 0
2020-02-01 1 1
2020-02-01 2 0
2020-02-01 3 1
2020-02-01 4 0
2020-03-01 1 0
2020-03-01 2 1
2020-03-01 3 1
2020-03-01 4 0
2020-04-01 1 0
2020-04-01 2 0
2020-04-01 3 0
2020-04-01 4 0
2020-05-01 1 0
2020-05-01 2 0
2020-05-01 3 0
2020-05-01 4 0

View File

@ -0,0 +1,81 @@
DROP TABLE IF EXISTS with_fill_date;
CREATE TABLE with_fill_date (d Date, d32 Date32) ENGINE = Memory;
INSERT INTO with_fill_date VALUES (toDate('2020-02-05'), toDate32('2020-02-05'));
INSERT INTO with_fill_date VALUES (toDate('2020-02-16'), toDate32('2020-02-16'));
INSERT INTO with_fill_date VALUES (toDate('2020-03-03'), toDate32('2020-03-03'));
INSERT INTO with_fill_date VALUES (toDate('2020-06-10'), toDate32('2020-06-10'));
SELECT '1 DAY';
SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 DAY LIMIT 5;
SELECT '1 WEEK';
SELECT toStartOfWeek(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 WEEK LIMIT 5;
SELECT '1 MONTH';
SELECT toStartOfMonth(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 MONTH LIMIT 5;
SELECT '3 MONTH';
SELECT toStartOfMonth(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL
FROM toDate('2020-01-01')
TO toDate('2021-01-01')
STEP INTERVAL 3 MONTH;
SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 HOUR LIMIT 5; -- { serverError 475 }
SELECT '1 DAY';
SELECT d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 DAY LIMIT 5;
SELECT '1 WEEK';
SELECT toStartOfWeek(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 WEEK LIMIT 5;
SELECT '1 MONTH';
SELECT toStartOfMonth(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL STEP INTERVAL 1 MONTH LIMIT 5;
SELECT '3 MONTH';
SELECT toStartOfMonth(d32) as d32, count() FROM with_fill_date GROUP BY d32 ORDER BY d32 WITH FILL
FROM toDate('2020-01-01')
TO toDate('2021-01-01')
STEP INTERVAL 3 MONTH;
SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 1 HOUR LIMIT 5; -- { serverError 475 }
DROP TABLE with_fill_date;
DROP TABLE IF EXISTS with_fill_date;
CREATE TABLE with_fill_date (d DateTime, d64 DateTime64) ENGINE = Memory;
INSERT INTO with_fill_date VALUES (toDateTime('2020-02-05 10:20:00'), toDateTime64('2020-02-05 10:20:00', 3));
INSERT INTO with_fill_date VALUES (toDateTime('2020-03-08 11:01:00'), toDateTime64('2020-03-08 11:01:00', 3));
SELECT '15 MINUTE';
SELECT d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 15 MINUTE LIMIT 5;
SELECT '6 HOUR';
SELECT toStartOfHour(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 6 HOUR LIMIT 5;
SELECT '10 DAY';
SELECT toStartOfDay(d) as d, count() FROM with_fill_date GROUP BY d ORDER BY d WITH FILL STEP INTERVAL 10 DAY LIMIT 5;
SELECT '15 MINUTE';
SELECT d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 15 MINUTE LIMIT 5;
SELECT '6 HOUR';
SELECT toStartOfHour(d64) as d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 6 HOUR LIMIT 5;
SELECT '10 DAY';
SELECT toStartOfDay(d64) as d64, count() FROM with_fill_date GROUP BY d64 ORDER BY d64 WITH FILL STEP INTERVAL 10 DAY LIMIT 5;
DROP TABLE with_fill_date;
SELECT number FROM numbers(100) ORDER BY number WITH FILL STEP INTERVAL 1 HOUR; -- { serverError 475 }
CREATE TABLE with_fill_date (d Date, id UInt32) ENGINE = Memory;
INSERT INTO with_fill_date VALUES (toDate('2020-02-05'), 1);
INSERT INTO with_fill_date VALUES (toDate('2020-02-16'), 3);
INSERT INTO with_fill_date VALUES (toDate('2020-03-10'), 2);
INSERT INTO with_fill_date VALUES (toDate('2020-03-03'), 3);
SELECT '1 MONTH';
SELECT toStartOfMonth(d) as d, id, count() FROM with_fill_date
GROUP BY d, id
ORDER BY
d WITH FILL
FROM toDate('2020-01-01')
TO toDate('2020-05-01')
STEP INTERVAL 1 MONTH,
id WITH FILL FROM 1 TO 5;
DROP TABLE with_fill_date;