diff --git a/debian/control b/debian/control index 45d2836b4a0..633b7e7c8a3 100644 --- a/debian/control +++ b/debian/control @@ -5,12 +5,9 @@ Maintainer: Alexey Milovidov Build-Depends: debhelper (>= 9), cmake | cmake3, ninja-build, - clang-11, - llvm-11, + clang-13, + llvm-13, libc6-dev, - libicu-dev, - libreadline-dev, - gperf, tzdata Standards-Version: 3.9.8 diff --git a/docs/en/commercial/cloud.md b/docs/en/commercial/cloud.md index c6ed80d4fdb..afa2e23b7a8 100644 --- a/docs/en/commercial/cloud.md +++ b/docs/en/commercial/cloud.md @@ -6,4 +6,4 @@ toc_title: Cloud # ClickHouse Cloud Service {#clickhouse-cloud-service} !!! info "Info" - Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/docs/en/commercial/support.md b/docs/en/commercial/support.md index 8ee976c8d6f..33b69b40b2d 100644 --- a/docs/en/commercial/support.md +++ b/docs/en/commercial/support.md @@ -6,4 +6,4 @@ toc_title: Support # ClickHouse Commercial Support Service {#clickhouse-commercial-support-service} !!! info "Info" - Detailed public description for ClickHouse support services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse support services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index a22d2031803..d188b4bb147 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -114,15 +114,25 @@ To do so, create the `/Library/LaunchDaemons/limit.maxfiles.plist` file with the ``` -Execute the following command: +Give the file correct permissions: ``` bash sudo chown root:wheel /Library/LaunchDaemons/limit.maxfiles.plist ``` -Reboot. +Validate that the file is correct: -To check if it’s working, you can use `ulimit -n` command. +``` bash +plutil /Library/LaunchDaemons/limit.maxfiles.plist +``` + +Load the file (or reboot): + +``` bash +sudo launchctl load -w /Library/LaunchDaemons/limit.maxfiles.plist +``` + +To check if it’s working, use the `ulimit -n` or `launchctl limit maxfiles` commands. ## Run ClickHouse server: diff --git a/docs/ja/commercial/cloud.md b/docs/ja/commercial/cloud.md index dceffcd591f..312b8aed6ea 100644 --- a/docs/ja/commercial/cloud.md +++ b/docs/ja/commercial/cloud.md @@ -8,4 +8,4 @@ toc_title: "\u30AF\u30E9\u30A6\u30C9" # ClickHouse Cloud Service {#clickhouse-cloud-service} !!! info "Info" - Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/docs/ru/commercial/cloud.md b/docs/ru/commercial/cloud.md index 2bdb8d68da5..549978f3cdf 100644 --- a/docs/ru/commercial/cloud.md +++ b/docs/ru/commercial/cloud.md @@ -6,4 +6,4 @@ toc_title: "Поставщики облачных услуг ClickHouse" # Поставщики облачных услуг ClickHouse {#clickhouse-cloud-service-providers} !!! info "Info" - Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/docs/zh/commercial/cloud.md b/docs/zh/commercial/cloud.md index e8c098db5be..61a4f638bbc 100644 --- a/docs/zh/commercial/cloud.md +++ b/docs/zh/commercial/cloud.md @@ -8,4 +8,4 @@ toc_title: 云 # ClickHouse Cloud Service {#clickhouse-cloud-service} !!! info "Info" - Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse cloud services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/docs/zh/commercial/support.md b/docs/zh/commercial/support.md index 5e5d00a22b8..a3714c98d44 100644 --- a/docs/zh/commercial/support.md +++ b/docs/zh/commercial/support.md @@ -6,4 +6,4 @@ toc_title: 支持 # ClickHouse 商业支持服务提供商 {#clickhouse-commercial-support-service-providers} !!! info "Info" - Detailed public description for ClickHouse support services is not ready yet, please [contact us](/company/#contact) to learn more. + Detailed public description for ClickHouse support services is not ready yet, please [contact us](https://clickhouse.com/company/#contact) to learn more. diff --git a/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp new file mode 100644 index 00000000000..8569e8f9c8c --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionExponentialMovingAverage.cpp @@ -0,0 +1,98 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + + +/** See the comments in ExponentiallySmoothedCounter.h + */ +class AggregateFunctionExponentialMovingAverage final + : public IAggregateFunctionDataHelper +{ +private: + String name; + Float64 half_decay; + +public: + AggregateFunctionExponentialMovingAverage(const DataTypes & argument_types_, const Array & params) + : IAggregateFunctionDataHelper(argument_types_, params) + { + if (params.size() != 1) + throw Exception{"Aggregate function " + getName() + " requires exactly one parameter: half decay time.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + + half_decay = applyVisitor(FieldVisitorConvertToNumber(), params[0]); + } + + String getName() const override + { + return "exponentialMovingAverage"; + } + + DataTypePtr getReturnType() const override + { + return std::make_shared>(); + } + + bool allocatesMemoryInArena() const override { return false; } + + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & value = columns[0]->getFloat64(row_num); + const auto & time = columns[1]->getFloat64(row_num); + this->data(place).add(value, time, half_decay); + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs), half_decay); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + { + writeBinary(this->data(place).value, buf); + writeBinary(this->data(place).time, buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + { + readBinary(this->data(place).value, buf); + readBinary(this->data(place).time, buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + auto & column = assert_cast &>(to); + column.getData().push_back(this->data(place).get(half_decay)); + } +}; + +void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory & factory) +{ + factory.registerFunction("exponentialMovingAverage", + [](const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) -> AggregateFunctionPtr + { + assertBinary(name, argument_types); + for (const auto & type : argument_types) + if (!isNumber(*type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Both arguments for aggregate function {} must have numeric type, got {}", name, type->getName()); + return std::make_shared(argument_types, params); + }); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index dd1f292a392..5d0af719290 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -50,7 +50,9 @@ void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); void registerAggregateFunctionSingleValueOrNull(AggregateFunctionFactory &); void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &); +void registerAggregateFunctionExponentialMovingAverage(AggregateFunctionFactory &); void registerAggregateFunctionSparkbar(AggregateFunctionFactory &); +void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -66,8 +68,6 @@ void registerAggregateFunctionCombinatorDistinct(AggregateFunctionCombinatorFact void registerWindowFunctions(AggregateFunctionFactory & factory); -void registerAggregateFunctionIntervalLengthSum(AggregateFunctionFactory &); - void registerAggregateFunctions() { { @@ -116,11 +116,11 @@ void registerAggregateFunctions() registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); registerAggregateFunctionSingleValueOrNull(factory); + registerAggregateFunctionIntervalLengthSum(factory); + registerAggregateFunctionExponentialMovingAverage(factory); + registerAggregateFunctionSparkbar(factory); registerWindowFunctions(factory); - - registerAggregateFunctionIntervalLengthSum(factory); - registerAggregateFunctionSparkbar(factory); } { diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 366e61bc8e2..fd02bde57bc 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -130,10 +130,16 @@ void Connection::connect(const ConnectionTimeouts & timeouts) } catch (Poco::TimeoutException & e) { + /// disconnect() will reset the socket, get timeouts before. + const std::string & message = fmt::format("{} ({}, receive timeout {} ms, send timeout {} ms)", + e.displayText(), getDescription(), + socket->getReceiveTimeout().totalMilliseconds(), + socket->getSendTimeout().totalMilliseconds()); + disconnect(); /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); + throw NetException(message, ErrorCodes::SOCKET_TIMEOUT); } } diff --git a/src/Common/ExponentiallySmoothedCounter.h b/src/Common/ExponentiallySmoothedCounter.h new file mode 100644 index 00000000000..28d4e5e25c1 --- /dev/null +++ b/src/Common/ExponentiallySmoothedCounter.h @@ -0,0 +1,114 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +/** https://en.wikipedia.org/wiki/Exponential_smoothing + * + * Exponentially smoothed average over time is weighted average with weight proportional to negative exponent of the time passed. + * For example, the last value is taken with weight 1/2, the value one second ago with weight 1/4, two seconds ago - 1/8, etc. + * It can be understood as an average over sliding window, but with different kernel. + * + * As an advantage, it is easy to update. Instead of collecting values and calculating a series of x1 / 2 + x2 / 4 + x3 / 8... + * just calculate x_old / 2 + x_new / 2. + * + * It is often used for resource usage metrics. For example, "load average" in Linux is exponentially smoothed moving average. + * We can use exponentially smoothed counters in query scheduler. + */ +struct ExponentiallySmoothedAverage +{ + /// The sum. It contains the last value and all previous values scaled accordingly to the difference of their time to the reference time. + /// Older values are summed with exponentially smaller coefficients. + /// To obtain the average, you have to divide this value to the sum of all coefficients (see 'sumWeights'). + + double value = 0; + + /// The point of reference. You can translate the value to a different point of reference (see 'remap'). + /// You can imagine that the value exponentially decays over time. + /// But it is also meaningful to treat the whole counters as constants over time but in another non-linear coordinate system, + /// that inflates over time, while the counter itself does not change + /// (it continues to be the same physical quantity, but only changes its representation in the "usual" coordinate system). + + /// Recap: the whole counter is one dimensional and it can be represented as a curve formed by two dependent coordinates in 2d plane, + /// the space can be represented by (value, time) coordinates, and the curves will be exponentially decaying over time, + /// alternatively the space can be represented by (exponentially_adjusted_value, time) and then the curves will be constant over time. + + /// Also useful analogy is the exponential representation of a number: x = a * exp(b) = a * e (where e = exp(b)) + /// a number x is represented by a curve in 2d plane that can be parametrized by coordinates (a, b) or (a, e). + + double time = 0; + + + ExponentiallySmoothedAverage() + { + } + + ExponentiallySmoothedAverage(double current_value, double current_time) + : value(current_value), time(current_time) + { + } + + /// How much value decays after time_passed. + static double scale(double time_passed, double half_decay_time) + { + return exp2(-time_passed / half_decay_time); + } + + /// Sum of weights of all values. Divide by it to get the average. + static double sumWeights(double half_decay_time) + { + double k = scale(1.0, half_decay_time); + return 1 / (1 - k); + } + + /// Obtain the same counter in another point of reference. + ExponentiallySmoothedAverage remap(double current_time, double half_decay_time) const + { + return ExponentiallySmoothedAverage(value * scale(current_time - time, half_decay_time), current_time); + } + + /// Merge two counters. It is done by moving to the same point of reference and summing the values. + static ExponentiallySmoothedAverage merge(const ExponentiallySmoothedAverage & a, const ExponentiallySmoothedAverage & b, double half_decay_time) + { + if (a.time > b.time) + return ExponentiallySmoothedAverage(a.value + b.remap(a.time, half_decay_time).value, a.time); + if (a.time < b.time) + return ExponentiallySmoothedAverage(b.value + a.remap(b.time, half_decay_time).value, b.time); + + return ExponentiallySmoothedAverage(a.value + b.value, a.time); + } + + void merge(const ExponentiallySmoothedAverage & other, double half_decay_time) + { + *this = merge(*this, other, half_decay_time); + } + + void add(double new_value, double current_time, double half_decay_time) + { + merge(ExponentiallySmoothedAverage(new_value, current_time), half_decay_time); + } + + /// Calculate the average from the sum. + double get(double half_decay_time) const + { + return value / sumWeights(half_decay_time); + } + + double get(double current_time, double half_decay_time) const + { + return remap(current_time, half_decay_time).get(half_decay_time); + } + + /// Compare two counters (by moving to the same point of reference and comparing sums). + /// You can store the counters in container and sort it without changing the stored values over time. + bool less(const ExponentiallySmoothedAverage & other, double half_decay_time) const + { + return remap(other.time, half_decay_time).value < other.value; + } +}; + +} diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 50e0fad0265..527b68e623a 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -54,7 +54,9 @@ bool ReadBufferFromPocoSocket::nextImpl() } catch (const Poco::TimeoutException &) { - throw NetException("Timeout exceeded while reading from socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT); + throw NetException(fmt::format("Timeout exceeded while reading from socket ({}, {} ms)", + peer_address.toString(), + socket.impl()->getReceiveTimeout().totalMilliseconds()), ErrorCodes::SOCKET_TIMEOUT); } catch (const Poco::IOException & e) { diff --git a/src/IO/WriteBufferFromPocoSocket.cpp b/src/IO/WriteBufferFromPocoSocket.cpp index a0e4de4c831..79c8952f5a3 100644 --- a/src/IO/WriteBufferFromPocoSocket.cpp +++ b/src/IO/WriteBufferFromPocoSocket.cpp @@ -57,7 +57,9 @@ void WriteBufferFromPocoSocket::nextImpl() } catch (const Poco::TimeoutException &) { - throw NetException("Timeout exceeded while writing to socket (" + peer_address.toString() + ")", ErrorCodes::SOCKET_TIMEOUT); + throw NetException(fmt::format("Timeout exceeded while writing to socket ({}, {} ms)", + peer_address.toString(), + socket.impl()->getSendTimeout().totalMilliseconds()), ErrorCodes::SOCKET_TIMEOUT); } catch (const Poco::IOException & e) { diff --git a/tests/performance/sum.xml b/tests/performance/sum.xml index 075102998c6..57b879a360d 100644 --- a/tests/performance/sum.xml +++ b/tests/performance/sum.xml @@ -24,7 +24,7 @@ FROM numbers_mt(200000000) SETTINGS max_threads = 8 - SELECT sum(x) FROM nullfloat32 - SELECT sum(x::Nullable(Float64)) FROM nullfloat32 + SELECT sum(x) FROM nullfloat32 + SELECT sum(x::Nullable(Float64)) FROM nullfloat32 DROP TABLE IF EXISTS nullfloat32 diff --git a/tests/queries/0_stateless/2020_exponential_smoothing.reference b/tests/queries/0_stateless/2020_exponential_smoothing.reference new file mode 100644 index 00000000000..8ebf4c3c066 --- /dev/null +++ b/tests/queries/0_stateless/2020_exponential_smoothing.reference @@ -0,0 +1,130 @@ +1 0 0.5 +0 1 0.25 +0 2 0.125 +0 3 0.0625 +0 4 0.03125 +0 5 0.015625 +0 6 0.0078125 +0 7 0.00390625 +0 8 0.001953125 +0 9 0.0009765625 +1 0 0.067 +0 1 0.062 +0 2 0.058 +0 3 0.054 +0 4 0.051 +0 5 0.047 +0 6 0.044 +0 7 0.041 +0 8 0.038 +0 9 0.036 +0 0 0 +1 1 0.5 +2 2 1.25 +3 3 2.125 +4 4 3.0625 +5 5 4.03125 +6 6 5.015625 +7 7 6.0078125 +8 8 7.00390625 +9 9 8.001953125 +1 0 0.067 ███▎ +0 1 0.062 ███ +0 2 0.058 ██▊ +0 3 0.054 ██▋ +0 4 0.051 ██▌ +0 5 0.047 ██▎ +0 6 0.044 ██▏ +0 7 0.041 ██ +0 8 0.038 █▊ +0 9 0.036 █▋ +0 10 0.033 █▋ +0 11 0.031 █▌ +0 12 0.029 █▍ +0 13 0.027 █▎ +0 14 0.025 █▎ +0 15 0.024 █▏ +0 16 0.022 █ +0 17 0.021 █ +0 18 0.019 ▊ +0 19 0.018 ▊ +0 20 0.017 ▋ +0 21 0.016 ▋ +0 22 0.015 ▋ +0 23 0.014 ▋ +0 24 0.013 ▋ +1 25 0.079 ███▊ +1 26 0.14 ███████ +1 27 0.198 █████████▊ +1 28 0.252 ████████████▌ +1 29 0.302 ███████████████ +1 30 0.349 █████████████████▍ +1 31 0.392 ███████████████████▌ +1 32 0.433 █████████████████████▋ +1 33 0.471 ███████████████████████▌ +1 34 0.506 █████████████████████████▎ +1 35 0.539 ██████████████████████████▊ +1 36 0.57 ████████████████████████████▌ +1 37 0.599 █████████████████████████████▊ +1 38 0.626 ███████████████████████████████▎ +1 39 0.651 ████████████████████████████████▌ +1 40 0.674 █████████████████████████████████▋ +1 41 0.696 ██████████████████████████████████▋ +1 42 0.716 ███████████████████████████████████▋ +1 43 0.735 ████████████████████████████████████▋ +1 44 0.753 █████████████████████████████████████▋ +1 45 0.77 ██████████████████████████████████████▍ +1 46 0.785 ███████████████████████████████████████▎ +1 47 0.8 ███████████████████████████████████████▊ +1 48 0.813 ████████████████████████████████████████▋ +1 49 0.825 █████████████████████████████████████████▎ +1 0 0.5 █████████████████████████ +0 1 0.25 ████████████▌ +0 2 0.125 ██████▎ +0 3 0.062 ███ +0 4 0.031 █▌ +1 5 0.516 █████████████████████████▋ +0 6 0.258 ████████████▊ +0 7 0.129 ██████▍ +0 8 0.064 ███▏ +0 9 0.032 █▌ +1 10 0.516 █████████████████████████▋ +0 11 0.258 ████████████▊ +0 12 0.129 ██████▍ +0 13 0.065 ███▏ +0 14 0.032 █▌ +1 15 0.516 █████████████████████████▋ +0 16 0.258 ████████████▊ +0 17 0.129 ██████▍ +0 18 0.065 ███▏ +0 19 0.032 █▌ +1 20 0.516 █████████████████████████▋ +0 21 0.258 ████████████▊ +0 22 0.129 ██████▍ +0 23 0.065 ███▏ +0 24 0.032 █▌ +1 25 0.516 █████████████████████████▋ +0 26 0.258 ████████████▊ +0 27 0.129 ██████▍ +0 28 0.065 ███▏ +0 29 0.032 █▌ +1 30 0.516 █████████████████████████▋ +0 31 0.258 ████████████▊ +0 32 0.129 ██████▍ +0 33 0.065 ███▏ +0 34 0.032 █▌ +1 35 0.516 █████████████████████████▋ +0 36 0.258 ████████████▊ +0 37 0.129 ██████▍ +0 38 0.065 ███▏ +0 39 0.032 █▌ +1 40 0.516 █████████████████████████▋ +0 41 0.258 ████████████▊ +0 42 0.129 ██████▍ +0 43 0.065 ███▏ +0 44 0.032 █▌ +1 45 0.516 █████████████████████████▋ +0 46 0.258 ████████████▊ +0 47 0.129 ██████▍ +0 48 0.065 ███▏ +0 49 0.032 █▌ diff --git a/tests/queries/0_stateless/2020_exponential_smoothing.sql b/tests/queries/0_stateless/2020_exponential_smoothing.sql new file mode 100644 index 00000000000..a210225453a --- /dev/null +++ b/tests/queries/0_stateless/2020_exponential_smoothing.sql @@ -0,0 +1,32 @@ +SELECT number = 0 AS value, number AS time, exponentialMovingAverage(1)(value, time) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS exp_smooth FROM numbers(10); +SELECT value, time, round(exp_smooth, 3) FROM (SELECT number = 0 AS value, number AS time, exponentialMovingAverage(10)(value, time) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS exp_smooth FROM numbers(10)); + +SELECT number AS value, number AS time, exponentialMovingAverage(1)(value, time) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS exp_smooth FROM numbers(10); + +SELECT + value, + time, + round(exp_smooth, 3), + bar(exp_smooth, 0, 1, 50) AS bar +FROM +( + SELECT + (number = 0) OR (number >= 25) AS value, + number AS time, + exponentialMovingAverage(10)(value, time) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS exp_smooth + FROM numbers(50) +); + +SELECT + value, + time, + round(exp_smooth, 3), + bar(exp_smooth, 0, 1, 50) AS bar +FROM +( + SELECT + (number % 5) = 0 AS value, + number AS time, + exponentialMovingAverage(1)(value, time) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS exp_smooth + FROM numbers(50) +); diff --git a/tests/queries/0_stateless/2021_exponential_sum.reference b/tests/queries/0_stateless/2021_exponential_sum.reference new file mode 100644 index 00000000000..5bd77479cf7 --- /dev/null +++ b/tests/queries/0_stateless/2021_exponential_sum.reference @@ -0,0 +1,8 @@ +0.0009765625 +0.0009775171065493646 +0.0009775171065493646 +0.0009775171065493646 +0.0009775171065493646 +0.0009775171065493646 +0.0009775171065493646 +0.0009775171065493646 diff --git a/tests/queries/0_stateless/2021_exponential_sum.sql b/tests/queries/0_stateless/2021_exponential_sum.sql new file mode 100644 index 00000000000..8ab7638029c --- /dev/null +++ b/tests/queries/0_stateless/2021_exponential_sum.sql @@ -0,0 +1,9 @@ +-- Check that it is deterministic +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(10); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(100); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(1000); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(10000); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(100000); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(1000000); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(10000000); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM numbers_mt(100000000); diff --git a/tests/queries/0_stateless/2021_exponential_sum_shard.reference b/tests/queries/0_stateless/2021_exponential_sum_shard.reference new file mode 100644 index 00000000000..8453706a05a --- /dev/null +++ b/tests/queries/0_stateless/2021_exponential_sum_shard.reference @@ -0,0 +1,5 @@ +0.009775171065493644 +0.009775171065493644 +0.009775171065493644 +0.009775171065493644 +0.009775171065493644 diff --git a/tests/queries/0_stateless/2021_exponential_sum_shard.sql b/tests/queries/0_stateless/2021_exponential_sum_shard.sql new file mode 100644 index 00000000000..49fde0fe217 --- /dev/null +++ b/tests/queries/0_stateless/2021_exponential_sum_shard.sql @@ -0,0 +1,6 @@ +-- Check that it is deterministic +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(1000)); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000)); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(100000)); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(1000000)); +WITH number % 10 = 0 AS value, number AS time SELECT exponentialMovingAverage(1)(value, time) AS exp_smooth FROM remote('127.0.0.{1..10}', numbers_mt(10000000));