未验证 提交 c5dac3d4 编写于 作者: A alexey-milovidov 提交者: GitHub

Merge pull request #3724 from ClickHouse-Ninja/master

timeSlots: add optional parameter 'Size'
......@@ -8,6 +8,7 @@
#include <Functions/FunctionHelpers.h>
#include <Functions/extractTimeZoneFromFunctionArguments.h>
#include <IO/WriteHelpers.h>
namespace DB
{
......@@ -30,10 +31,8 @@ namespace ErrorCodes
template <typename DurationType>
struct TimeSlotsImpl
{
static constexpr UInt32 TIME_SLOT_SIZE = 1800;
static void vector_vector(
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations,
const PaddedPODArray<UInt32> & starts, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = starts.size();
......@@ -44,9 +43,9 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + durations[i]) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + durations[i]) / time_slot_size; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
......@@ -55,7 +54,7 @@ struct TimeSlotsImpl
}
static void vector_constant(
const PaddedPODArray<UInt32> & starts, DurationType duration,
const PaddedPODArray<UInt32> & starts, DurationType duration, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = starts.size();
......@@ -66,9 +65,9 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = starts[i] / TIME_SLOT_SIZE; value <= (starts[i] + duration) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = starts[i] / time_slot_size; value <= (starts[i] + duration) / time_slot_size; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
......@@ -77,7 +76,7 @@ struct TimeSlotsImpl
}
static void constant_vector(
UInt32 start, const PaddedPODArray<DurationType> & durations,
UInt32 start, const PaddedPODArray<DurationType> & durations, UInt32 time_slot_size,
PaddedPODArray<UInt32> & result_values, ColumnArray::Offsets & result_offsets)
{
size_t size = durations.size();
......@@ -88,9 +87,9 @@ struct TimeSlotsImpl
ColumnArray::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + durations[i]) / TIME_SLOT_SIZE; ++value)
for (UInt32 value = start / time_slot_size; value <= (start + durations[i]) / time_slot_size; ++value)
{
result_values.push_back(value * TIME_SLOT_SIZE);
result_values.push_back(value * time_slot_size);
++current_offset;
}
......@@ -99,11 +98,11 @@ struct TimeSlotsImpl
}
static void constant_constant(
UInt32 start, DurationType duration,
UInt32 start, DurationType duration, UInt32 time_slot_size,
Array & result)
{
for (UInt32 value = start / TIME_SLOT_SIZE; value <= (start + duration) / TIME_SLOT_SIZE; ++value)
result.push_back(value * TIME_SLOT_SIZE);
for (UInt32 value = start / time_slot_size; value <= (start + duration) / time_slot_size; ++value)
result.push_back(value * time_slot_size);
}
};
......@@ -112,6 +111,7 @@ class FunctionTimeSlots : public IFunction
{
public:
static constexpr auto name = "timeSlots";
static constexpr UInt32 TIME_SLOT_SIZE = 1800;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionTimeSlots>(); }
String getName() const override
......@@ -119,10 +119,16 @@ public:
return name;
}
size_t getNumberOfArguments() const override { return 2; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 2 && arguments.size() != 3)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2 or 3",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!WhichDataType(arguments[0].type).isDateTime())
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
......@@ -131,9 +137,13 @@ public:
throw Exception("Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName() + ". Must be UInt32.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isUInt32())
throw Exception("Illegal type " + arguments[1].type->getName() + " of third argument of function " + getName() + ". Must be UInt32.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
/// If time zone is specified for source data type, attach it to the resulting type.
/// Note that there is no explicit time zone argument for this function (we specify 2 as an argument number with explicit time zone).
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)));
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)));
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
......@@ -147,30 +157,43 @@ public:
auto res = ColumnArray::create(ColumnUInt32::create());
ColumnUInt32::Container & res_values = typeid_cast<ColumnUInt32 &>(res->getData()).getData();
auto time_slot_size = TIME_SLOT_SIZE;
if (arguments.size() == 3)
{
auto time_slot_column = checkAndGetColumnConst<ColumnUInt32>(block.getByPosition(arguments[2]).column.get());
if (!time_slot_column)
throw Exception("Third argument for function " + getName() + " must be constant UInt32", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (time_slot_size = time_slot_column->getValue<UInt32>(); time_slot_size == 0)
throw Exception("Third argument for function " + getName() + " must be greater than zero", ErrorCodes::ILLEGAL_COLUMN);
}
if (starts && durations)
{
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vector_vector(starts->getData(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (starts && const_durations)
{
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::vector_constant(starts->getData(), const_durations->getValue<UInt32>(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (const_starts && durations)
{
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), res_values, res->getOffsets());
TimeSlotsImpl<UInt32>::constant_vector(const_starts->getValue<UInt32>(), durations->getData(), time_slot_size, res_values, res->getOffsets());
block.getByPosition(result).column = std::move(res);
}
else if (const_starts && const_durations)
{
Array const_res;
TimeSlotsImpl<UInt32>::constant_constant(const_starts->getValue<UInt32>(), const_durations->getValue<UInt32>(), const_res);
TimeSlotsImpl<UInt32>::constant_constant(const_starts->getValue<UInt32>(), const_durations->getValue<UInt32>(), time_slot_size, const_res);
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(input_rows_count, const_res);
}
else
throw Exception("Illegal columns " + block.getByPosition(arguments[0]).column->getName()
+ ", " + block.getByPosition(arguments[1]).column->getName()
+ ", " + block.getByPosition(arguments[2]).column->getName()
+ " of arguments of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
......
2000-01-02 03:00:00
['2000-01-02 03:00:00','2000-01-02 03:30:00','2000-01-02 04:00:00','2000-01-02 04:30:00','2000-01-02 05:00:00','2000-01-02 05:30:00']
['2000-01-02 03:00:00','2000-01-02 03:10:00','2000-01-02 03:20:00','2000-01-02 03:30:00','2000-01-02 03:40:00','2000-01-02 03:50:00','2000-01-02 04:00:00','2000-01-02 04:10:00','2000-01-02 04:20:00','2000-01-02 04:30:00','2000-01-02 04:40:00','2000-01-02 04:50:00','2000-01-02 05:00:00','2000-01-02 05:10:00','2000-01-02 05:20:00','2000-01-02 05:30:00','2000-01-02 05:40:00','2000-01-02 05:50:00']
['2000-01-02 03:04:00','2000-01-02 03:04:30','2000-01-02 03:05:00','2000-01-02 03:05:30','2000-01-02 03:06:00','2000-01-02 03:06:30','2000-01-02 03:07:00','2000-01-02 03:07:30','2000-01-02 03:08:00','2000-01-02 03:08:30','2000-01-02 03:09:00','2000-01-02 03:09:30','2000-01-02 03:10:00','2000-01-02 03:10:30','2000-01-02 03:11:00','2000-01-02 03:11:30','2000-01-02 03:12:00','2000-01-02 03:12:30','2000-01-02 03:13:00','2000-01-02 03:13:30','2000-01-02 03:14:00']
SELECT timeSlot(toDateTime('2000-01-02 03:04:05', 'UTC'));
SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000));
SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(10000), toUInt32(600));
SELECT timeSlots(toDateTime('2000-01-02 03:04:05', 'UTC'), toUInt32(600), toUInt32(30));
\ No newline at end of file
......@@ -148,9 +148,9 @@ The same as 'today() - 1'.
Rounds the time to the half hour.
This function is specific to Yandex.Metrica, since half an hour is the minimum amount of time for breaking a session into two sessions if a tracking tag shows a single user's consecutive pageviews that differ in time by strictly more than this amount. This means that tuples (the tag ID, user ID, and time slot) can be used to search for pageviews that are included in the corresponding session.
## timeSlots(StartTime, Duration)
## timeSlots(StartTime, Duration,\[, Size\])
For a time interval starting at 'StartTime' and continuing for 'Duration' seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the half hour.
For a time interval starting at 'StartTime' and continuing for 'Duration' seconds, it returns an array of moments in time, consisting of points from this interval rounded down to the 'Size' in seconds. 'Size' is an optional parameter: a constant UInt32, set to 1800 by default.
For example, `timeSlots(toDateTime('2012-01-01 12:20:00'), 600) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`.
This is necessary for searching for pageviews in the corresponding session.
......
......@@ -121,8 +121,9 @@ SELECT
Округляет время до получаса.
Эта функция является специфичной для Яндекс.Метрики, так как пол часа - минимальное время, для которого, если соседние по времени хиты одного посетителя на одном счётчике отстоят друг от друга строго более, чем на это время, визит может быть разбит на два визита. То есть, кортежи (номер счётчика, идентификатор посетителя, тайм-слот) могут использоваться для поиска хитов, входящий в соответствующий визит.
## timeSlots(StartTime, Duration)
Для интервала времени, начинающегося в StartTime и продолжающегося Duration секунд, возвращает массив моментов времени, состоящий из округлений вниз до получаса точек из этого интервала.
## timeSlots(StartTime, Duration,\[, Size\])
Для интервала времени, начинающегося в 'StartTime' и продолжающегося 'Duration' секунд, возвращает массив моментов времени, состоящий из округлений вниз до 'Size' точек в секундах из этого интервала. 'Size' - необязательный параметр, константный UInt32, по умолчанию равен 1800.
Например, `timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)) = [toDateTime('2012-01-01 12:00:00'), toDateTime('2012-01-01 12:30:00')]`.
Это нужно для поиска хитов, входящих в соответствующий визит.
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册