2020-01-14 03:08:54 +00:00
|
|
|
#pragma once
|
|
|
|
|
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
2020-02-02 03:54:54 +00:00
|
|
|
#include <DataTypes/DataTypeDateTime.h>
|
2020-01-14 03:08:54 +00:00
|
|
|
#include <DataTypes/DataTypeInterval.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
2020-02-02 03:54:54 +00:00
|
|
|
#include <Functions/extractTimeZoneFromFunctionArguments.h>
|
2020-01-14 03:08:54 +00:00
|
|
|
#include <IO/WriteHelpers.h>
|
2020-02-02 03:54:54 +00:00
|
|
|
#include <common/DateLUT.h>
|
2020-06-08 06:27:30 +00:00
|
|
|
#include <numeric>
|
2020-01-14 03:08:54 +00:00
|
|
|
|
|
|
|
#include "IFunctionImpl.h"
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
/** Window functions:
|
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* TUMBLE(time_attr, interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
|
|
|
* TUMBLE_START(window_id)
|
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* TUMBLE_START(time_attr, interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
|
|
|
* TUMBLE_END(window_id)
|
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* TUMBLE_END(time_attr, interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* HOP(time_attr, hop_interval, window_interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
|
|
|
* HOP_START(window_id)
|
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* HOP_START(time_attr, hop_interval, window_interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
|
|
|
* HOP_END(window_id)
|
|
|
|
*
|
2020-02-02 03:54:54 +00:00
|
|
|
* HOP_END(time_attr, hop_interval, window_interval [, timezone])
|
2020-01-14 03:08:54 +00:00
|
|
|
*
|
|
|
|
*/
|
2020-01-14 16:24:26 +00:00
|
|
|
enum WindowFunctionName
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
TUMBLE,
|
|
|
|
TUMBLE_START,
|
|
|
|
TUMBLE_END,
|
|
|
|
HOP,
|
|
|
|
HOP_START,
|
2020-06-17 15:06:19 +00:00
|
|
|
HOP_END,
|
|
|
|
WINDOW_ID
|
2020-01-14 03:08:54 +00:00
|
|
|
};
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
|
|
|
extern const int ARGUMENT_OUT_OF_BOUND;
|
|
|
|
}
|
|
|
|
|
|
|
|
template <IntervalKind::Kind unit>
|
|
|
|
struct ToStartOfTransform;
|
|
|
|
|
|
|
|
#define TRANSFORM_DATE(INTERVAL_KIND) \
|
|
|
|
template <> \
|
|
|
|
struct ToStartOfTransform<IntervalKind::INTERVAL_KIND> \
|
|
|
|
{ \
|
|
|
|
static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \
|
|
|
|
{ \
|
|
|
|
return time_zone.toStartOf##INTERVAL_KIND##Interval(time_zone.toDayNum(t), delta); \
|
|
|
|
} \
|
|
|
|
};
|
|
|
|
TRANSFORM_DATE(Year)
|
|
|
|
TRANSFORM_DATE(Quarter)
|
|
|
|
TRANSFORM_DATE(Month)
|
|
|
|
TRANSFORM_DATE(Week)
|
|
|
|
TRANSFORM_DATE(Day)
|
|
|
|
#undef TRANSFORM_DATE
|
|
|
|
|
|
|
|
#define TRANSFORM_TIME(INTERVAL_KIND) \
|
|
|
|
template <> \
|
|
|
|
struct ToStartOfTransform<IntervalKind::INTERVAL_KIND> \
|
|
|
|
{ \
|
|
|
|
static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \
|
|
|
|
{ \
|
|
|
|
return time_zone.toStartOf##INTERVAL_KIND##Interval(t, delta); \
|
|
|
|
} \
|
|
|
|
};
|
|
|
|
TRANSFORM_TIME(Hour)
|
|
|
|
TRANSFORM_TIME(Minute)
|
|
|
|
TRANSFORM_TIME(Second)
|
|
|
|
#undef TRANSFORM_DATE
|
|
|
|
|
|
|
|
template <IntervalKind::Kind unit>
|
|
|
|
struct AddTime;
|
|
|
|
|
|
|
|
#define ADD_DATE(INTERVAL_KIND) \
|
|
|
|
template <> \
|
|
|
|
struct AddTime<IntervalKind::INTERVAL_KIND> \
|
|
|
|
{ \
|
|
|
|
static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone) { return time_zone.add##INTERVAL_KIND##s(t, delta); } \
|
|
|
|
};
|
|
|
|
ADD_DATE(Year)
|
|
|
|
ADD_DATE(Quarter)
|
|
|
|
ADD_DATE(Month)
|
|
|
|
ADD_DATE(Week)
|
|
|
|
ADD_DATE(Day)
|
|
|
|
#undef ADD_DATE
|
|
|
|
|
|
|
|
#define ADD_TIME(INTERVAL_KIND, INTERVAL) \
|
|
|
|
template <> \
|
|
|
|
struct AddTime<IntervalKind::INTERVAL_KIND> \
|
|
|
|
{ \
|
|
|
|
static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) { return t + INTERVAL * delta; } \
|
|
|
|
};
|
|
|
|
ADD_TIME(Hour, 3600)
|
|
|
|
ADD_TIME(Minute, 60)
|
|
|
|
ADD_TIME(Second, 1)
|
|
|
|
#undef ADD_TIME
|
|
|
|
|
|
|
|
namespace
|
|
|
|
{
|
|
|
|
static std::tuple<IntervalKind::Kind, Int64>
|
|
|
|
dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name)
|
|
|
|
{
|
|
|
|
const auto * interval_type = checkAndGetDataType<DataTypeInterval>(interval_column.type.get());
|
|
|
|
if (!interval_type)
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
const auto * interval_column_const_int64 = checkAndGetColumnConst<ColumnInt64>(interval_column.column.get());
|
|
|
|
if (!interval_column_const_int64)
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
Int64 num_units = interval_column_const_int64->getValue<Int64>();
|
|
|
|
if (num_units <= 0)
|
|
|
|
throw Exception(
|
|
|
|
"Value for column " + interval_column.name + " of function " + function_name + " must be positive.",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
|
|
|
|
|
|
|
return {interval_type->getKind(), num_units};
|
|
|
|
}
|
|
|
|
|
|
|
|
static ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name)
|
|
|
|
{
|
|
|
|
if (const ColumnTuple * col_tuple = checkAndGetColumn<ColumnTuple>(column.get()); col_tuple)
|
|
|
|
{
|
|
|
|
if (!checkColumn<ColumnVector<UInt32>>(*col_tuple->getColumnPtr(index)))
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column for first argument of function " + function_name + ". Must be a Tuple(DataTime, DataTime)",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
return col_tuple->getColumnPtr(index);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception(
|
2020-06-17 15:22:17 +00:00
|
|
|
"Illegal column for first argument of function " + function_name + ". Must be Tuple",
|
2020-01-14 03:08:54 +00:00
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <WindowFunctionName type>
|
|
|
|
struct WindowImpl
|
|
|
|
{
|
|
|
|
static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name);
|
|
|
|
};
|
|
|
|
|
|
|
|
template <>
|
|
|
|
struct WindowImpl<TUMBLE>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "TUMBLE";
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() != 2 && arguments.size() != 3)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size())
|
|
|
|
+ ", should be 2.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
if (!WhichDataType(arguments[0].type).isDateTime())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[1].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name
|
|
|
|
+ ". This argument is optional and must be a constant string with timezone name",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-01-14 03:08:54 +00:00
|
|
|
return std::make_shared<DataTypeTuple>(DataTypes{std::make_shared<DataTypeDateTime>(), std::make_shared<DataTypeDateTime>()});
|
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto & interval_column = block.getByPosition(arguments[1]);
|
|
|
|
const auto & from_datatype = *time_column.type.get();
|
|
|
|
const auto which_type = WhichDataType(from_datatype);
|
|
|
|
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
|
2020-02-02 03:54:54 +00:00
|
|
|
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0);
|
2020-01-14 03:08:54 +00:00
|
|
|
if (!which_type.isDateTime() || !time_column_vec)
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
auto interval = dispatchForIntervalColumns(interval_column, function_name);
|
|
|
|
|
|
|
|
switch (std::get<0>(interval))
|
|
|
|
{
|
|
|
|
case IntervalKind::Second:
|
|
|
|
return execute_tumble<IntervalKind::Second>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Minute:
|
|
|
|
return execute_tumble<IntervalKind::Minute>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Hour:
|
|
|
|
return execute_tumble<IntervalKind::Hour>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Day:
|
|
|
|
return execute_tumble<IntervalKind::Day>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Week:
|
|
|
|
return execute_tumble<IntervalKind::Week>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Month:
|
|
|
|
return execute_tumble<IntervalKind::Month>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Quarter:
|
|
|
|
return execute_tumble<IntervalKind::Quarter>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
case IntervalKind::Year:
|
|
|
|
return execute_tumble<IntervalKind::Year>(*time_column_vec, std::get<1>(interval), time_zone);
|
|
|
|
}
|
|
|
|
__builtin_unreachable();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <IntervalKind::Kind unit>
|
|
|
|
static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone)
|
|
|
|
{
|
|
|
|
const auto & time_data = time_column.getData();
|
|
|
|
size_t size = time_column.size();
|
|
|
|
auto start = ColumnUInt32::create(size);
|
|
|
|
auto end = ColumnUInt32::create(size);
|
|
|
|
ColumnUInt32::Container & start_data = start->getData();
|
|
|
|
ColumnUInt32::Container & end_data = end->getData();
|
|
|
|
for (size_t i = 0; i != size; ++i)
|
|
|
|
{
|
2020-01-14 09:04:47 +00:00
|
|
|
UInt32 wid = static_cast<UInt32>(ToStartOfTransform<unit>::execute(time_data[i], num_units, time_zone));
|
2020-01-14 03:08:54 +00:00
|
|
|
start_data[i] = wid;
|
|
|
|
end_data[i] = AddTime<unit>::execute(wid, num_units, time_zone);
|
|
|
|
}
|
|
|
|
MutableColumns result;
|
|
|
|
result.emplace_back(std::move(start));
|
|
|
|
result.emplace_back(std::move(end));
|
|
|
|
return ColumnTuple::create(std::move(result));
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <>
|
|
|
|
struct WindowImpl<TUMBLE_START>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "TUMBLE_START";
|
|
|
|
|
|
|
|
static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
|
|
|
{
|
|
|
|
if (arguments.size() == 1)
|
|
|
|
{
|
2020-06-19 09:35:18 +00:00
|
|
|
auto type_ = WhichDataType(arguments[0].type);
|
|
|
|
if (!type_.isTuple() && !type_.isUInt32())
|
2020-01-14 03:08:54 +00:00
|
|
|
throw Exception(
|
2020-06-19 09:35:18 +00:00
|
|
|
"Illegal type of first argument of function " + function_name + " should be tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN);
|
2020-01-14 03:08:54 +00:00
|
|
|
return std::make_shared<DataTypeDateTime>();
|
|
|
|
}
|
2020-02-02 03:54:54 +00:00
|
|
|
else if (arguments.size() == 2 || arguments.size() == 3)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
if (!WhichDataType(arguments[0].type).isDateTime())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[1].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name
|
|
|
|
+ ". This argument is optional and must be a constant string with timezone name",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-01-14 03:08:54 +00:00
|
|
|
return std::make_shared<DataTypeDateTime>();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size())
|
|
|
|
+ ", should not larger than 2.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto which_type = WhichDataType(time_column.type);
|
|
|
|
ColumnPtr result_column_;
|
|
|
|
if (which_type.isDateTime())
|
2020-02-02 03:54:54 +00:00
|
|
|
result_column_ = WindowImpl<TUMBLE>::dispatchForColumns(block, arguments, function_name);
|
2020-01-14 03:08:54 +00:00
|
|
|
else
|
|
|
|
result_column_ = block.getByPosition(arguments[0]).column;
|
|
|
|
return executeWindowBound(result_column_, 0, function_name);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <>
|
|
|
|
struct WindowImpl<TUMBLE_END>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "TUMBLE_END";
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
2020-01-14 16:24:26 +00:00
|
|
|
return WindowImpl<TUMBLE_START>::getReturnType(arguments, function_name);
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto which_type = WhichDataType(time_column.type);
|
|
|
|
ColumnPtr result_column_;
|
|
|
|
if (which_type.isDateTime())
|
2020-02-02 03:54:54 +00:00
|
|
|
result_column_ = WindowImpl<TUMBLE>::dispatchForColumns(block, arguments, function_name);
|
2020-01-14 03:08:54 +00:00
|
|
|
else
|
|
|
|
result_column_ = block.getByPosition(arguments[0]).column;
|
|
|
|
return executeWindowBound(result_column_, 1, function_name);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <>
|
|
|
|
struct WindowImpl<HOP>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "HOP";
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() != 3 && arguments.size() != 4)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size())
|
|
|
|
+ ", should be 3.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
if (!WhichDataType(arguments[0].type).isDateTime())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[1].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[2].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name
|
|
|
|
+ ". This argument is optional and must be a constant string with timezone name",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-06-08 06:27:30 +00:00
|
|
|
return std::make_shared<DataTypeTuple>(DataTypes{std::make_shared<DataTypeDateTime>(), std::make_shared<DataTypeDateTime>()});
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto & hop_interval_column = block.getByPosition(arguments[1]);
|
|
|
|
const auto & window_interval_column = block.getByPosition(arguments[2]);
|
|
|
|
const auto & from_datatype = *time_column.type.get();
|
|
|
|
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
|
2020-02-02 03:54:54 +00:00
|
|
|
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 3, 0);
|
2020-01-14 03:08:54 +00:00
|
|
|
if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec)
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column " + time_column.name + " argument of function " + function_name
|
|
|
|
+ ". Must contain dates or dates with time",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name);
|
|
|
|
auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name);
|
|
|
|
|
|
|
|
if (std::get<0>(hop_interval) != std::get<0>(window_interval))
|
|
|
|
throw Exception(
|
|
|
|
"Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (std::get<1>(hop_interval) > std::get<1>(window_interval))
|
|
|
|
throw Exception(
|
|
|
|
"Value for hop interval of function " + function_name + " must not larger than window interval.",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
|
|
|
|
|
|
|
switch (std::get<0>(window_interval))
|
|
|
|
{
|
|
|
|
case IntervalKind::Second:
|
|
|
|
return execute_hop<IntervalKind::Second>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Minute:
|
|
|
|
return execute_hop<IntervalKind::Minute>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Hour:
|
|
|
|
return execute_hop<IntervalKind::Hour>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Day:
|
|
|
|
return execute_hop<IntervalKind::Day>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Week:
|
|
|
|
return execute_hop<IntervalKind::Week>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Month:
|
|
|
|
return execute_hop<IntervalKind::Month>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Quarter:
|
|
|
|
return execute_hop<IntervalKind::Quarter>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Year:
|
|
|
|
return execute_hop<IntervalKind::Year>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
}
|
|
|
|
__builtin_unreachable();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <IntervalKind::Kind kind>
|
|
|
|
static ColumnPtr
|
|
|
|
execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone)
|
|
|
|
{
|
|
|
|
const auto & time_data = time_column.getData();
|
|
|
|
size_t size = time_column.size();
|
|
|
|
|
2020-06-08 06:27:30 +00:00
|
|
|
auto start = ColumnUInt32::create(size);
|
|
|
|
auto end = ColumnUInt32::create(size);
|
2020-01-14 03:08:54 +00:00
|
|
|
ColumnUInt32::Container & start_data = start->getData();
|
|
|
|
ColumnUInt32::Container & end_data = end->getData();
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
2020-01-14 09:04:47 +00:00
|
|
|
UInt32 wstart = static_cast<UInt32>(ToStartOfTransform<kind>::execute(time_data[i], hop_num_units, time_zone));
|
2020-01-14 03:08:54 +00:00
|
|
|
UInt32 wend = AddTime<kind>::execute(wstart, hop_num_units, time_zone);
|
|
|
|
wstart = AddTime<kind>::execute(wend, -1 * window_num_units, time_zone);
|
|
|
|
|
|
|
|
UInt32 wend_ = wend;
|
|
|
|
UInt32 wend_latest;
|
|
|
|
|
|
|
|
do
|
|
|
|
{
|
|
|
|
wend_latest = wend_;
|
|
|
|
wend_ = AddTime<kind>::execute(wend_, -1 * hop_num_units, time_zone);
|
|
|
|
} while (wend_ > time_data[i]);
|
|
|
|
|
2020-06-08 06:27:30 +00:00
|
|
|
end_data[i] = wend_latest;
|
|
|
|
start_data[i] = AddTime<kind>::execute(wend_latest, -1 * window_num_units, time_zone);
|
|
|
|
}
|
|
|
|
MutableColumns result;
|
|
|
|
result.emplace_back(std::move(start));
|
|
|
|
result.emplace_back(std::move(end));
|
|
|
|
return ColumnTuple::create(std::move(result));
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
template <>
|
2020-06-17 15:06:19 +00:00
|
|
|
struct WindowImpl<WINDOW_ID>
|
2020-06-08 06:27:30 +00:00
|
|
|
{
|
2020-06-17 15:06:19 +00:00
|
|
|
static constexpr auto name = "WINDOW_ID";
|
2020-06-08 06:27:30 +00:00
|
|
|
|
|
|
|
[[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
|
|
|
{
|
|
|
|
if (!WhichDataType(arguments[0].type).isDateTime())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[1].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
2020-06-17 15:06:19 +00:00
|
|
|
if (arguments.size() >= 3 && !WhichDataType(arguments[2].type).isInterval())
|
2020-06-08 06:27:30 +00:00
|
|
|
throw Exception(
|
|
|
|
"Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name
|
|
|
|
+ ". This argument is optional and must be a constant string with timezone name",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-06-17 15:06:19 +00:00
|
|
|
if (arguments.size() > 4)
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size())
|
|
|
|
+ ", should not larger than 4.",
|
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2020-06-08 06:27:30 +00:00
|
|
|
return std::make_shared<DataTypeUInt32>();
|
|
|
|
}
|
|
|
|
|
2020-06-09 15:57:47 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-06-17 15:06:19 +00:00
|
|
|
dispatchForHopColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-06-08 06:27:30 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto & hop_interval_column = block.getByPosition(arguments[1]);
|
|
|
|
const auto & window_interval_column = block.getByPosition(arguments[2]);
|
|
|
|
const auto & from_datatype = *time_column.type.get();
|
|
|
|
const auto * time_column_vec = checkAndGetColumn<ColumnUInt32>(time_column.column.get());
|
|
|
|
const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 3, 0);
|
|
|
|
if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec)
|
|
|
|
throw Exception(
|
|
|
|
"Illegal column " + time_column.name + " argument of function " + function_name
|
|
|
|
+ ". Must contain dates or dates with time",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
|
|
|
auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name);
|
|
|
|
auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name);
|
|
|
|
|
|
|
|
if (std::get<0>(hop_interval) != std::get<0>(window_interval))
|
|
|
|
throw Exception(
|
|
|
|
"Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (std::get<1>(hop_interval) > std::get<1>(window_interval))
|
|
|
|
throw Exception(
|
|
|
|
"Value for hop interval of function " + function_name + " must not larger than window interval.",
|
|
|
|
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
|
|
|
|
|
|
|
switch (std::get<0>(window_interval))
|
|
|
|
{
|
|
|
|
case IntervalKind::Second:
|
|
|
|
return execute_hop_slice<IntervalKind::Second>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Minute:
|
|
|
|
return execute_hop_slice<IntervalKind::Minute>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Hour:
|
|
|
|
return execute_hop_slice<IntervalKind::Hour>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Day:
|
|
|
|
return execute_hop_slice<IntervalKind::Day>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Week:
|
|
|
|
return execute_hop_slice<IntervalKind::Week>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Month:
|
|
|
|
return execute_hop_slice<IntervalKind::Month>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Quarter:
|
|
|
|
return execute_hop_slice<IntervalKind::Quarter>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
case IntervalKind::Year:
|
|
|
|
return execute_hop_slice<IntervalKind::Year>(
|
|
|
|
*time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone);
|
|
|
|
}
|
|
|
|
__builtin_unreachable();
|
|
|
|
}
|
|
|
|
|
|
|
|
template <IntervalKind::Kind kind>
|
|
|
|
static ColumnPtr
|
|
|
|
execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone)
|
|
|
|
{
|
|
|
|
Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units);
|
|
|
|
|
|
|
|
const auto & time_data = time_column.getData();
|
|
|
|
size_t size = time_column.size();
|
|
|
|
|
|
|
|
auto end = ColumnUInt32::create(size);
|
|
|
|
ColumnUInt32::Container & end_data = end->getData();
|
|
|
|
for (size_t i = 0; i < size; ++i)
|
|
|
|
{
|
|
|
|
UInt32 wstart = static_cast<UInt32>(ToStartOfTransform<kind>::execute(time_data[i], hop_num_units, time_zone));
|
|
|
|
UInt32 wend = AddTime<kind>::execute(wstart, hop_num_units, time_zone);
|
|
|
|
|
|
|
|
UInt32 wend_ = wend;
|
|
|
|
UInt32 wend_latest;
|
2020-01-14 03:08:54 +00:00
|
|
|
|
2020-06-08 06:27:30 +00:00
|
|
|
do
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
2020-06-08 06:27:30 +00:00
|
|
|
wend_latest = wend_;
|
|
|
|
wend_ = AddTime<kind>::execute(wend_, -1 * gcd_num_units, time_zone);
|
|
|
|
} while (wend_ > time_data[i]);
|
|
|
|
|
|
|
|
end_data[i] = wend_latest;
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|
2020-06-08 06:27:30 +00:00
|
|
|
return end;
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|
2020-06-17 15:06:19 +00:00
|
|
|
|
|
|
|
[[maybe_unused]] static ColumnPtr
|
|
|
|
dispatchForTumbleColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
|
|
|
{
|
|
|
|
ColumnPtr column = WindowImpl<TUMBLE>::dispatchForColumns(block, arguments, function_name);
|
|
|
|
return executeWindowBound(column, 1, function_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
[[maybe_unused]] static ColumnPtr
|
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
|
|
|
{
|
2020-06-19 09:35:18 +00:00
|
|
|
if (arguments.size() == 2)
|
2020-06-17 15:06:19 +00:00
|
|
|
return dispatchForTumbleColumns(block, arguments, function_name);
|
|
|
|
else
|
2020-06-19 09:35:18 +00:00
|
|
|
{
|
|
|
|
const auto & third_column = block.getByPosition(arguments[2]);
|
|
|
|
if (arguments.size() == 3 && WhichDataType(third_column.type).isString())
|
|
|
|
return dispatchForTumbleColumns(block, arguments, function_name);
|
|
|
|
else
|
|
|
|
return dispatchForHopColumns(block, arguments, function_name);
|
|
|
|
}
|
2020-06-17 15:06:19 +00:00
|
|
|
}
|
2020-01-14 03:08:54 +00:00
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <>
|
|
|
|
struct WindowImpl<HOP_START>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "HOP_START";
|
|
|
|
|
|
|
|
static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
|
|
|
{
|
|
|
|
if (arguments.size() == 1)
|
|
|
|
{
|
|
|
|
auto type_ = WhichDataType(arguments[0].type);
|
2020-06-08 06:27:30 +00:00
|
|
|
if (!type_.isTuple() && !type_.isUInt32())
|
2020-01-14 03:08:54 +00:00
|
|
|
throw Exception(
|
2020-06-17 15:22:17 +00:00
|
|
|
"Illegal type of first argument of function " + function_name + " should be Tuple or UInt32",
|
2020-01-14 03:08:54 +00:00
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
return std::make_shared<DataTypeDateTime>();
|
|
|
|
}
|
2020-02-02 03:54:54 +00:00
|
|
|
else if (arguments.size() == 3 || arguments.size() == 4)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
if (!WhichDataType(arguments[0].type).isDateTime())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[1].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
if (!WhichDataType(arguments[2].type).isInterval())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN);
|
2020-02-02 03:54:54 +00:00
|
|
|
if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString())
|
|
|
|
throw Exception(
|
|
|
|
"Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name
|
|
|
|
+ ". This argument is optional and must be a constant string with timezone name",
|
|
|
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
2020-01-14 03:08:54 +00:00
|
|
|
return std::make_shared<DataTypeDateTime>();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
throw Exception(
|
|
|
|
"Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size())
|
2020-06-08 06:27:30 +00:00
|
|
|
+ ", should be 1, 3 or 4.",
|
2020-01-14 03:08:54 +00:00
|
|
|
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto which_type = WhichDataType(time_column.type);
|
|
|
|
ColumnPtr result_column_;
|
2020-06-08 06:27:30 +00:00
|
|
|
if (arguments.size() == 1)
|
|
|
|
{
|
|
|
|
if (which_type.isUInt32())
|
|
|
|
return time_column.column;
|
|
|
|
else //isTuple
|
|
|
|
result_column_ = time_column.column;
|
|
|
|
}
|
2020-01-14 03:08:54 +00:00
|
|
|
else
|
2020-06-08 06:27:30 +00:00
|
|
|
result_column_ = WindowImpl<HOP>::dispatchForColumns(block, arguments, function_name);
|
2020-01-14 03:08:54 +00:00
|
|
|
return executeWindowBound(result_column_, 0, function_name);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <>
|
|
|
|
struct WindowImpl<HOP_END>
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
static constexpr auto name = "HOP_END";
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
2020-01-14 16:24:26 +00:00
|
|
|
return WindowImpl<HOP_START>::getReturnType(arguments, function_name);
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
[[maybe_unused]] static ColumnPtr
|
2020-02-02 03:54:54 +00:00
|
|
|
dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name)
|
2020-01-14 03:08:54 +00:00
|
|
|
{
|
|
|
|
const auto & time_column = block.getByPosition(arguments[0]);
|
|
|
|
const auto which_type = WhichDataType(time_column.type);
|
|
|
|
ColumnPtr result_column_;
|
2020-06-08 06:27:30 +00:00
|
|
|
if (arguments.size() == 1)
|
|
|
|
{
|
|
|
|
if (which_type.isUInt32())
|
|
|
|
return time_column.column;
|
|
|
|
else //isTuple
|
|
|
|
result_column_ = time_column.column;
|
|
|
|
}
|
2020-01-14 03:08:54 +00:00
|
|
|
else
|
2020-06-08 06:27:30 +00:00
|
|
|
result_column_ = WindowImpl<HOP>::dispatchForColumns(block, arguments, function_name);
|
|
|
|
|
2020-01-14 03:08:54 +00:00
|
|
|
return executeWindowBound(result_column_, 1, function_name);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
template <WindowFunctionName type>
|
2020-01-14 03:08:54 +00:00
|
|
|
class FunctionWindow : public IFunction
|
|
|
|
{
|
|
|
|
public:
|
2020-01-14 16:24:26 +00:00
|
|
|
static constexpr auto name = WindowImpl<type>::name;
|
2020-01-14 03:08:54 +00:00
|
|
|
static FunctionPtr create(const Context &) { return std::make_shared<FunctionWindow>(); }
|
|
|
|
String getName() const override { return name; }
|
2020-02-02 03:54:54 +00:00
|
|
|
bool isVariadic() const override { return true; }
|
|
|
|
size_t getNumberOfArguments() const override { return 0; }
|
2020-01-14 03:08:54 +00:00
|
|
|
bool useDefaultImplementationForConstants() const override { return true; }
|
2020-02-02 03:54:54 +00:00
|
|
|
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; }
|
2020-01-14 03:08:54 +00:00
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return WindowImpl<type>::getReturnType(arguments, name); }
|
2020-01-14 03:08:54 +00:00
|
|
|
|
|
|
|
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
|
|
|
{
|
2020-02-02 03:54:54 +00:00
|
|
|
auto result_column = WindowImpl<type>::dispatchForColumns(block, arguments, name);
|
2020-01-14 03:08:54 +00:00
|
|
|
block.getByPosition(result).column = std::move(result_column);
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
2020-01-14 16:24:26 +00:00
|
|
|
using FunctionTumble = FunctionWindow<TUMBLE>;
|
|
|
|
using FunctionTumbleStart = FunctionWindow<TUMBLE_START>;
|
|
|
|
using FunctionTumbleEnd = FunctionWindow<TUMBLE_END>;
|
|
|
|
using FunctionHop = FunctionWindow<HOP>;
|
2020-06-17 15:06:19 +00:00
|
|
|
using FunctionWindowId = FunctionWindow<WINDOW_ID>;
|
2020-01-14 16:24:26 +00:00
|
|
|
using FunctionHopStart = FunctionWindow<HOP_START>;
|
|
|
|
using FunctionHopEnd = FunctionWindow<HOP_END>;
|
2020-01-14 03:08:54 +00:00
|
|
|
}
|