Revert "Non Negative Derivative window function"

This commit is contained in:
Alexey Milovidov 2022-06-29 08:56:15 +03:00 committed by GitHub
parent f5d26572df
commit dea3b5bfce
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 5 additions and 266 deletions

View File

@ -9,13 +9,13 @@ namespace ErrorCodes
extern const int SYNTAX_ERROR;
}
Float64 IntervalKind::toAvgSeconds() const
Int32 IntervalKind::toAvgSeconds() const
{
switch (kind)
{
case IntervalKind::Nanosecond: return 0.000000001;
case IntervalKind::Microsecond: return 0.000001;
case IntervalKind::Millisecond: return 0.001;
case IntervalKind::Nanosecond:
case IntervalKind::Microsecond:
case IntervalKind::Millisecond: return 0; /// fractional parts of seconds have 0 seconds
case IntervalKind::Second: return 1;
case IntervalKind::Minute: return 60;
case IntervalKind::Hour: return 3600;
@ -28,25 +28,6 @@ Float64 IntervalKind::toAvgSeconds() const
__builtin_unreachable();
}
bool IntervalKind::isFixedLength() const
{
switch (kind)
{
case IntervalKind::Nanosecond:
case IntervalKind::Microsecond:
case IntervalKind::Millisecond:
case IntervalKind::Second:
case IntervalKind::Minute:
case IntervalKind::Hour:
case IntervalKind::Day:
case IntervalKind::Week: return true;
case IntervalKind::Month:
case IntervalKind::Quarter:
case IntervalKind::Year: return false;
}
__builtin_unreachable();
}
IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds)
{
if (num_seconds)

View File

@ -31,15 +31,12 @@ struct IntervalKind
/// Returns number of seconds in one interval.
/// For `Month`, `Quarter` and `Year` the function returns an average number of seconds.
Float64 toAvgSeconds() const;
Int32 toAvgSeconds() const;
/// Chooses an interval kind based on number of seconds.
/// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Hour`.
static IntervalKind fromAvgSeconds(Int64 num_seconds);
/// Returns whether IntervalKind has a fixed number of seconds (e.g. Day) or non-fixed(e.g. Month)
bool isFixedLength() const;
/// Returns an uppercased version of what `toString()` returns.
const char * toKeyword() const;

View File

@ -13,7 +13,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/getLeastSupertype.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeInterval.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/convertFieldToType.h>
@ -28,7 +27,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int NOT_IMPLEMENTED;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
// Interface for true window functions. It's not much of an interface, they just
@ -2202,109 +2200,6 @@ struct WindowFunctionNthValue final : public WindowFunction
}
};
struct NonNegativeDerivativeState
{
Float64 previous_metric = 0;
Float64 previous_timestamp = 0;
};
// nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL 1 SECOND])
struct WindowFunctionNonNegativeDerivative final : public StatefulWindowFunction<NonNegativeDerivativeState>
{
static constexpr size_t ARGUMENT_METRIC = 0;
static constexpr size_t ARGUMENT_TIMESTAMP = 1;
static constexpr size_t ARGUMENT_INTERVAL = 2;
WindowFunctionNonNegativeDerivative(const std::string & name_,
const DataTypes & argument_types_, const Array & parameters_)
: StatefulWindowFunction(name_, argument_types_, parameters_)
{
if (!parameters.empty())
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function {} cannot be parameterized", name_);
}
if (argument_types.size() != 2 && argument_types.size() != 3)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Function {} takes 2 or 3 arguments", name_);
}
if (!isNumber(argument_types[ARGUMENT_METRIC]))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Argument {} must be a number, '{}' given",
ARGUMENT_METRIC,
argument_types[ARGUMENT_METRIC]->getName());
}
if (!isDateTime(argument_types[ARGUMENT_TIMESTAMP]) && !isDateTime64(argument_types[ARGUMENT_TIMESTAMP]))
{
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Argument {} must be DateTime or DateTime64, '{}' given",
ARGUMENT_TIMESTAMP,
argument_types[ARGUMENT_TIMESTAMP]->getName());
}
if (argument_types.size() == 3)
{
const DataTypeInterval * interval_datatype = checkAndGetDataType<DataTypeInterval>(argument_types[ARGUMENT_INTERVAL].get());
if (!interval_datatype)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Argument {} must be an INTERVAL, '{}' given",
ARGUMENT_INTERVAL,
argument_types[ARGUMENT_INTERVAL]->getName());
}
if (!interval_datatype->getKind().isFixedLength())
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The INTERVAL must be a week or shorter, '{}' given",
argument_types[ARGUMENT_INTERVAL]->getName());
}
interval_length = interval_datatype->getKind().toAvgSeconds();
interval_specified = true;
}
}
DataTypePtr getReturnType() const override { return argument_types[0]; }
bool allocatesMemoryInArena() const override { return false; }
void windowInsertResultInto(const WindowTransform * transform,
size_t function_index) override
{
const auto & current_block = transform->blockAt(transform->current_row);
const auto & workspace = transform->workspaces[function_index];
auto & state = getState(workspace);
auto interval_duration = interval_specified ? interval_length *
(*current_block.input_columns[workspace.argument_column_indices[ARGUMENT_INTERVAL]]).getFloat64(0) : 1;
Float64 last_metric = state.previous_metric;
Float64 last_timestamp = state.previous_timestamp;
Float64 curr_metric = WindowFunctionHelpers::getValue<Float64>(transform, function_index, ARGUMENT_METRIC, transform->current_row);
Float64 curr_timestamp = WindowFunctionHelpers::getValue<Float64>(transform, function_index, ARGUMENT_TIMESTAMP, transform->current_row);
Float64 time_elapsed = curr_timestamp - last_timestamp;
Float64 metric_diff = curr_metric - last_metric;
Float64 result = (time_elapsed != 0) ? (metric_diff / time_elapsed * interval_duration) : 0;
state.previous_metric = curr_metric;
state.previous_timestamp = curr_timestamp;
WindowFunctionHelpers::setValueToOutputColumn<Float64>(transform, function_index, result >= 0 ? result : 0);
}
private:
Float64 interval_length = 1;
bool interval_specified = false;
};
void registerWindowFunctions(AggregateFunctionFactory & factory)
{
@ -2404,13 +2299,6 @@ void registerWindowFunctions(AggregateFunctionFactory & factory)
return std::make_shared<WindowFunctionExponentialTimeDecayedAvg>(
name, argument_types, parameters);
}, properties});
factory.registerFunction("nonNegativeDerivative", {[](const std::string & name,
const DataTypes & argument_types, const Array & parameters, const Settings *)
{
return std::make_shared<WindowFunctionNonNegativeDerivative>(
name, argument_types, parameters);
}, properties});
}
}

View File

@ -1,64 +0,0 @@
1
1979-12-12 21:21:21.123 1.1 3.5045052519931732e-9
1979-12-12 21:21:22.000 1.3345 0.26738883339230357
1979-12-12 21:21:23.000 1.54 0.20550000000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 453.33916989529325
1979-12-12 21:21:21.123 1.1 1.0513515755979521e-17
1979-12-12 21:21:22.000 1.3345 8.021665001769108e-10
1979-12-12 21:21:23.000 1.54 6.165000000000001e-10
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 0.0000013600175096858798
1979-12-12 21:21:21.123 1.1 1.4018021007972692e-14
1979-12-12 21:21:22.000 1.3345 0.0000010695553335692141
1979-12-12 21:21:23.000 1.54 8.22e-7
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 0.001813356679581173
1979-12-12 21:21:21.123 1.1 1.7522526259965866e-11
1979-12-12 21:21:22.000 1.3345 0.0013369441669615178
1979-12-12 21:21:23.000 1.54 0.0010275000000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 2.2666958494764664
1979-12-12 21:21:21.123 1.1 2.102703151195904e-8
1979-12-12 21:21:22.000 1.3345 1.6043330003538214
1979-12-12 21:21:23.000 1.54 1.233
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 2720.0350193717595
1979-12-12 21:21:21.123 1.1 0.0000014718922058371327
1979-12-12 21:21:22.000 1.3345 112.3033100247675
1979-12-12 21:21:23.000 1.54 86.31
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:21.124 2.34 0
1979-12-12 21:21:21.127 3.7 190402.45135602317
1979-12-12 21:21:21.123 1.1 0.0001009297512574034
1979-12-12 21:21:21.124 2.34 35712459.78375156
1979-12-12 21:21:21.127 3.7 13056168.092984445
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 5918.400000000001
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.123 1.1 0.0027251032839498914
1979-12-12 21:21:21.124 2.34 964236414.1612921
1979-12-12 21:21:21.127 3.7 352516538.51058006
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 159796.80000000002
1979-12-12 21:21:23.000 1.54 0
1979-12-12 21:21:21.123 1.1 0.021195247764054712
1979-12-12 21:21:21.124 2.34 7499616554.587828
1979-12-12 21:21:21.127 3.7 2741795299.5267334
1979-12-12 21:21:21.129 2.1 0
1979-12-12 21:21:22.000 1.3345 0
1979-12-12 21:21:23.000 1.54 1242864
1979-12-12 21:21:23.000 1.54 0

View File

@ -1,63 +0,0 @@
DROP TABLE IF EXISTS nnd;
CREATE TABLE nnd
(
id Int8, ts DateTime64(3, 'UTC'), metric Float64
)
ENGINE=MergeTree()
ORDER BY id;
INSERT INTO nnd VALUES (1, toDateTime64('1979-12-12 21:21:21.123', 3, 'UTC'), 1.1), (2, toDateTime64('1979-12-12 21:21:21.124', 3, 'UTC'), 2.34), (3, toDateTime64('1979-12-12 21:21:21.127', 3, 'UTC'), 3.7);
INSERT INTO nnd VALUES (4, toDateTime64('1979-12-12 21:21:21.129', 3, 'UTC'), 2.1), (5, toDateTime('1979-12-12 21:21:22', 'UTC'), 1.3345), (6, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54), (7, toDateTime('1979-12-12 21:21:23', 'UTC'), 1.54);
-- shall work for precise intervals
-- INTERVAL 1 SECOND shall be default
SELECT (
SELECT
ts,
metric,
nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv
FROM nnd
LIMIT 5, 1
) = (
SELECT
ts,
metric,
nonNegativeDerivative(metric, ts, toIntervalSecond(1)) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv
FROM nnd
LIMIT 5, 1
);
SELECT ts, metric, nonNegativeDerivative(metric, ts) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Nanosecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Microsecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 4 MICROSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Millisecond
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 5 MILLISECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Second
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 6 SECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Minute
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 7 MINUTE) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Hour
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 8 HOUR) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Day
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 9 DAY) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- Week
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 10 WEEK) OVER (ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd;
-- shall not work for month, quarter, year (intervals with floating number of seconds)
-- Month
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 11 MONTH) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Quarter
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 12 QUARTER) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- Year
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 13 YEAR) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
-- test against wrong arguments/types
SELECT ts, metric, nonNegativeDerivative(metric, 1, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative('string not datetime', ts, INTERVAL 3 NANOSECOND) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric, ts, INTERVAL 3 NANOSECOND, id) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
SELECT ts, metric, nonNegativeDerivative(metric) OVER (PARTITION BY metric ORDER BY ts ASC Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS deriv FROM nnd; -- { serverError BAD_ARGUMENTS }
-- cleanup
DROP TABLE IF EXISTS nnd;