From 855e3378b6fac2cce52e5c89e5de0a5bfd018de6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 16:01:29 +0300 Subject: [PATCH 01/22] Added system.zeros and system.zeros_mt --- .../Storages/System/StorageSystemZeros.cpp | 132 ++++++++++++++++++ dbms/src/Storages/System/StorageSystemZeros.h | 43 ++++++ .../Storages/System/attachSystemTables.cpp | 3 + 3 files changed, 178 insertions(+) create mode 100644 dbms/src/Storages/System/StorageSystemZeros.cpp create mode 100644 dbms/src/Storages/System/StorageSystemZeros.h diff --git a/dbms/src/Storages/System/StorageSystemZeros.cpp b/dbms/src/Storages/System/StorageSystemZeros.cpp new file mode 100644 index 00000000000..e1f0745ff1b --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemZeros.cpp @@ -0,0 +1,132 @@ +#include + +#include +#include + +#include +#include + +namespace DB +{ + +namespace +{ + +struct ZerosState +{ + std::atomic num_generated_rows = 0; +}; + +using ZerosStatePtr = std::shared_ptr; + + +/// Source which generates zeros. +/// Uses state to share the number of generated rows between threads. +/// If state is nullptr, then limit is ignored. +class ZerosSource : public SourceWithProgress +{ +public: + ZerosSource(UInt64 block_size, UInt64 limit_, ZerosStatePtr state_) + : SourceWithProgress(createHeader()), limit(limit_), state(std::move(state_)) + { + column = createColumn(block_size); + } + + String getName() const override { return "Zeros"; } + +protected: + Chunk generate() override + { + auto column_ptr = column; + size_t column_size = column_ptr->size(); + + if (state) + { + auto generated_rows = state->num_generated_rows.fetch_add(column_size, std::memory_order_acquire); + + if (generated_rows >= limit) + return {}; + + if (generated_rows + column_size > limit) + { + column_size = limit - generated_rows; + column_ptr = createColumn(column_size); + } + } + + progress({column->size(), column->byteSize()}); + + return { Columns {std::move(column_ptr)}, column_size }; + } + +private: + UInt64 limit; + ZerosStatePtr state; + ColumnPtr column; + + static Block createHeader() + { + return { ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "zero") }; + } + + static ColumnPtr createColumn(size_t size) + { + auto column_ptr = ColumnUInt8::create(); + /// It is probably the fastest method to create zero column, cause resize_fill uses memset internally. + column_ptr->getData().resize_fill(size); + + return column_ptr; + } +}; + +} + +StorageSystemZeros::StorageSystemZeros(const std::string & name_, bool multithreaded_, std::optional limit_) + : IStorage({"system", name_}), multithreaded(multithreaded_), limit(limit_) +{ + setColumns(ColumnsDescription({{"zeros", std::make_shared()}})); +} + +Pipes StorageSystemZeros::read( + const Names & column_names, + const SelectQueryInfo &, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned num_streams) +{ + check(column_names); + + bool use_multiple_streams = multithreaded; + + if (limit && *limit < max_block_size) + { + max_block_size = static_cast(*limit); + use_multiple_streams = false; + } + + if (!use_multiple_streams) + num_streams = 1; + + Pipes res; + res.reserve(num_streams); + + ZerosStatePtr state; + + if (limit) + state = std::make_shared(); + + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared(max_block_size, *limit, state); + + if (limit && i == 0) + source->addTotalRowsApprox(*limit); + + res.emplace_back(std::move(source)); + } + + return res; +} + +} diff --git a/dbms/src/Storages/System/StorageSystemZeros.h b/dbms/src/Storages/System/StorageSystemZeros.h new file mode 100644 index 00000000000..36032608637 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemZeros.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/** Implements a table engine for the system table "zeros". + * The table contains the only column zero UInt8. + * From this table, you can read non-materialized zeros. + * + * You could also specify a limit (how many zeros to give). + * If multithreaded is specified, zeros will be generated in several streams. + */ +class StorageSystemZeros : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; +public: + std::string getName() const override { return "SystemZeros"; } + + Pipes read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + bool hasEvenlyDistributedRead() const override { return true; } + +private: + bool multithreaded; + std::optional limit; + +protected: + /// If even_distribution is true, numbers are distributed evenly between streams. + /// Otherwise, streams concurrently increment atomic. + StorageSystemZeros(const std::string & name_, bool multithreaded_, std::optional limit_ = std::nullopt); +}; + +} diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 06f00783384..db76d68f091 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -39,6 +39,7 @@ #include #include #include +#include #ifdef OS_LINUX #include @@ -53,6 +54,8 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("one", StorageSystemOne::create("one")); system_database.attachTable("numbers", StorageSystemNumbers::create("numbers", false)); system_database.attachTable("numbers_mt", StorageSystemNumbers::create("numbers_mt", true)); + system_database.attachTable("zeros", StorageSystemZeros::create("zeros", false)); + system_database.attachTable("zeros_mt", StorageSystemZeros::create("zeros_mt", true)); system_database.attachTable("databases", StorageSystemDatabases::create("databases")); system_database.attachTable("tables", StorageSystemTables::create("tables")); system_database.attachTable("columns", StorageSystemColumns::create("columns")); From 80b2060fa28bf4bad370da9e1cd674042ae7f8ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 16:14:49 +0300 Subject: [PATCH 02/22] Added table functions zeros and zeros_mt --- dbms/src/TableFunctions/TableFunctionZeros.cpp | 5 +++++ dbms/src/TableFunctions/TableFunctionZeros.h | 15 +++++++++++++++ 2 files changed, 20 insertions(+) create mode 100644 dbms/src/TableFunctions/TableFunctionZeros.cpp create mode 100644 dbms/src/TableFunctions/TableFunctionZeros.h diff --git a/dbms/src/TableFunctions/TableFunctionZeros.cpp b/dbms/src/TableFunctions/TableFunctionZeros.cpp new file mode 100644 index 00000000000..23f08ed795c --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionZeros.cpp @@ -0,0 +1,5 @@ +// +// Created by nik-kochetov on 3/10/20. +// + +#include "TableFunctionZeros.h" diff --git a/dbms/src/TableFunctions/TableFunctionZeros.h b/dbms/src/TableFunctions/TableFunctionZeros.h new file mode 100644 index 00000000000..8cb9ae0e6cb --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionZeros.h @@ -0,0 +1,15 @@ +// +// Created by nik-kochetov on 3/10/20. +// + +#ifndef CLICKHOUSE_TABLEFUNCTIONZEROS_H +#define CLICKHOUSE_TABLEFUNCTIONZEROS_H + + +class TableFunctionZeros +{ + +}; + + +#endif //CLICKHOUSE_TABLEFUNCTIONZEROS_H From d1073dd9dd17e422ef43ec873b150c4785bdeca4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 17:54:22 +0300 Subject: [PATCH 03/22] Added table functions zeros and zeros_mt --- dbms/src/Access/AccessType.h | 1 + .../src/TableFunctions/TableFunctionZeros.cpp | 59 +++++++++++++++++-- dbms/src/TableFunctions/TableFunctionZeros.h | 26 +++++--- .../TableFunctions/registerTableFunctions.cpp | 1 + .../TableFunctions/registerTableFunctions.h | 1 + 5 files changed, 77 insertions(+), 11 deletions(-) diff --git a/dbms/src/Access/AccessType.h b/dbms/src/Access/AccessType.h index 4b65f949885..6811bca93ae 100644 --- a/dbms/src/Access/AccessType.h +++ b/dbms/src/Access/AccessType.h @@ -129,6 +129,7 @@ enum class AccessType input, values, numbers, + zeros, merge, remote, mysql, diff --git a/dbms/src/TableFunctions/TableFunctionZeros.cpp b/dbms/src/TableFunctions/TableFunctionZeros.cpp index 23f08ed795c..79f49dc5d28 100644 --- a/dbms/src/TableFunctions/TableFunctionZeros.cpp +++ b/dbms/src/TableFunctions/TableFunctionZeros.cpp @@ -1,5 +1,56 @@ -// -// Created by nik-kochetov on 3/10/20. -// +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerTableFunctions.h" -#include "TableFunctionZeros.h" + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +template +StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +{ + if (const auto * function = ast_function->as()) + { + auto arguments = function->arguments->children; + + if (arguments.size() != 1) + throw Exception("Table function '" + getName() + "' requires 'length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + + UInt64 length = evaluateArgument(context, arguments[0]); + + context.checkAccess(AccessType::zeros); + + auto res = StorageSystemZeros::create(table_name, multithreaded, length); + res->startup(); + return res; + } + throw Exception("Table function '" + getName() + "' requires 'limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); +} + +void registerTableFunctionZeros(TableFunctionFactory & factory) +{ + factory.registerFunction>(); + factory.registerFunction>(); +} + +template +UInt64 TableFunctionZeros::evaluateArgument(const Context & context, ASTPtr & argument) const +{ + return evaluateConstantExpressionOrIdentifierAsLiteral(argument, context)->as().value.safeGet(); +} + +} diff --git a/dbms/src/TableFunctions/TableFunctionZeros.h b/dbms/src/TableFunctions/TableFunctionZeros.h index 8cb9ae0e6cb..b702d2bde63 100644 --- a/dbms/src/TableFunctions/TableFunctionZeros.h +++ b/dbms/src/TableFunctions/TableFunctionZeros.h @@ -1,15 +1,27 @@ -// -// Created by nik-kochetov on 3/10/20. -// +#pragma once -#ifndef CLICKHOUSE_TABLEFUNCTIONZEROS_H -#define CLICKHOUSE_TABLEFUNCTIONZEROS_H +#include +#include -class TableFunctionZeros +namespace DB { +/* zeros(limit), zeros_mt(limit) + * - the same as SELECT number FROM system.zeros LIMIT limit. + * Used for testing purposes, as a simple example of table function. + */ +template +class TableFunctionZeros : public ITableFunction +{ +public: + static constexpr auto name = multithreaded ? "zeros_mt" : "zeros"; + std::string getName() const override { return name; } +private: + StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; + + UInt64 evaluateArgument(const Context & context, ASTPtr & argument) const; }; -#endif //CLICKHOUSE_TABLEFUNCTIONZEROS_H +} diff --git a/dbms/src/TableFunctions/registerTableFunctions.cpp b/dbms/src/TableFunctions/registerTableFunctions.cpp index b9a96532b13..662f7b58d48 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.cpp +++ b/dbms/src/TableFunctions/registerTableFunctions.cpp @@ -11,6 +11,7 @@ void registerTableFunctions() registerTableFunctionMerge(factory); registerTableFunctionRemote(factory); registerTableFunctionNumbers(factory); + registerTableFunctionZeros(factory); registerTableFunctionFile(factory); registerTableFunctionURL(factory); registerTableFunctionValues(factory); diff --git a/dbms/src/TableFunctions/registerTableFunctions.h b/dbms/src/TableFunctions/registerTableFunctions.h index e1be5477c25..3cde28685cc 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.h +++ b/dbms/src/TableFunctions/registerTableFunctions.h @@ -8,6 +8,7 @@ class TableFunctionFactory; void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionRemote(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); +void registerTableFunctionZeros(TableFunctionFactory & factory); void registerTableFunctionFile(TableFunctionFactory & factory); void registerTableFunctionURL(TableFunctionFactory & factory); void registerTableFunctionValues(TableFunctionFactory & factory); From 6334c8f2abfdaa8b5ecc54e68ae53926f2e3dfa0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 20:02:15 +0300 Subject: [PATCH 04/22] Fix access type. --- dbms/src/Access/AccessType.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Access/AccessType.h b/dbms/src/Access/AccessType.h index 6811bca93ae..205840eecdf 100644 --- a/dbms/src/Access/AccessType.h +++ b/dbms/src/Access/AccessType.h @@ -289,6 +289,7 @@ namespace impl ACCESS_TYPE_TO_KEYWORD_CASE(input); ACCESS_TYPE_TO_KEYWORD_CASE(values); ACCESS_TYPE_TO_KEYWORD_CASE(numbers); + ACCESS_TYPE_TO_KEYWORD_CASE(zeros); ACCESS_TYPE_TO_KEYWORD_CASE(merge); ACCESS_TYPE_TO_KEYWORD_CASE(remote); ACCESS_TYPE_TO_KEYWORD_CASE(mysql); From 55ce0a9e8a3655f2af2a6e7ec0afc446b755932e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 20:27:21 +0300 Subject: [PATCH 05/22] Fix access type. --- dbms/src/Access/AccessFlags.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Access/AccessFlags.h b/dbms/src/Access/AccessFlags.h index f35c88f9831..5af804ddc48 100644 --- a/dbms/src/Access/AccessFlags.h +++ b/dbms/src/Access/AccessFlags.h @@ -401,6 +401,7 @@ private: auto input = std::make_unique("input()", next_flag++, GLOBAL_LEVEL); auto values = std::make_unique("values()", next_flag++, GLOBAL_LEVEL); auto numbers = std::make_unique("numbers()", next_flag++, GLOBAL_LEVEL); + auto zeros = std::make_unique("zeros()", next_flag++, GLOBAL_LEVEL); auto merge = std::make_unique("merge()", next_flag++, DATABASE_LEVEL); auto remote = std::make_unique("remote()", next_flag++, GLOBAL_LEVEL); ext::push_back(remote->aliases, "remoteSecure()", "cluster()"); @@ -409,7 +410,7 @@ private: auto jdbc = std::make_unique("jdbc()", next_flag++, GLOBAL_LEVEL); auto hdfs = std::make_unique("hdfs()", next_flag++, GLOBAL_LEVEL); auto s3 = std::make_unique("s3()", next_flag++, GLOBAL_LEVEL); - auto table_functions = std::make_unique("TABLE FUNCTIONS", std::move(file), std::move(url), std::move(input), std::move(values), std::move(numbers), std::move(merge), std::move(remote), std::move(mysql), std::move(odbc), std::move(jdbc), std::move(hdfs), std::move(s3)); + auto table_functions = std::make_unique("TABLE FUNCTIONS", std::move(file), std::move(url), std::move(input), std::move(values), std::move(numbers), std::move(zeros), std::move(merge), std::move(remote), std::move(mysql), std::move(odbc), std::move(jdbc), std::move(hdfs), std::move(s3)); ext::push_back(all, std::move(table_functions)); flags_to_keyword_tree_ = std::make_unique("ALL", std::move(all)); From 3042ff5db5bc42cd288cb6edabf9475f49f63c46 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 20:40:33 +0300 Subject: [PATCH 06/22] Fix zeors column name. --- dbms/src/Storages/System/StorageSystemZeros.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemZeros.cpp b/dbms/src/Storages/System/StorageSystemZeros.cpp index e1f0745ff1b..f721c9e1cf4 100644 --- a/dbms/src/Storages/System/StorageSystemZeros.cpp +++ b/dbms/src/Storages/System/StorageSystemZeros.cpp @@ -84,7 +84,7 @@ private: StorageSystemZeros::StorageSystemZeros(const std::string & name_, bool multithreaded_, std::optional limit_) : IStorage({"system", name_}), multithreaded(multithreaded_), limit(limit_) { - setColumns(ColumnsDescription({{"zeros", std::make_shared()}})); + setColumns(ColumnsDescription({{"zero", std::make_shared()}})); } Pipes StorageSystemZeros::read( From f052b03ee954c07c9f3d9ab749d1b34873599e94 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 20:42:31 +0300 Subject: [PATCH 07/22] Fix zeors column type. --- dbms/src/Storages/System/StorageSystemZeros.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemZeros.cpp b/dbms/src/Storages/System/StorageSystemZeros.cpp index f721c9e1cf4..ae9bf3d0902 100644 --- a/dbms/src/Storages/System/StorageSystemZeros.cpp +++ b/dbms/src/Storages/System/StorageSystemZeros.cpp @@ -66,7 +66,7 @@ private: static Block createHeader() { - return { ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "zero") }; + return { ColumnWithTypeAndName(ColumnUInt8::create(), std::make_shared(), "zero") }; } static ColumnPtr createColumn(size_t size) From 655ba5b0e7d915137836e286e9ab8ee27082ed77 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 10 Mar 2020 21:17:16 +0300 Subject: [PATCH 08/22] Fix comment. --- dbms/src/TableFunctions/TableFunctionZeros.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TableFunctions/TableFunctionZeros.h b/dbms/src/TableFunctions/TableFunctionZeros.h index b702d2bde63..c8f3cbabc0e 100644 --- a/dbms/src/TableFunctions/TableFunctionZeros.h +++ b/dbms/src/TableFunctions/TableFunctionZeros.h @@ -8,7 +8,7 @@ namespace DB { /* zeros(limit), zeros_mt(limit) - * - the same as SELECT number FROM system.zeros LIMIT limit. + * - the same as SELECT zero FROM system.zeros LIMIT limit. * Used for testing purposes, as a simple example of table function. */ template From 47a580e37b59400cf9d350623de8e31dcc48041f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 11 Mar 2020 12:31:49 +0300 Subject: [PATCH 09/22] Update README.md --- README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index a71ff01cc10..d9d0afc7005 100644 --- a/README.md +++ b/README.md @@ -14,5 +14,6 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events - * [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on March 28, 2020. - * [Talks on Saint HighLoad++ in St. Petersburg](https://www.highload.ru/spb/2020/abstracts/6647) on April 6, 2020. +* [Yandex C++ Open-Source Sprints in Moscow](https://events.yandex.ru/events/otkrytyj-kod-v-yandek-28-03-2020) on March 28, 2020. +* [ClickHouse Workshop in Novosibirsk](https://2020.codefest.ru/lecture/1628) on March 28, 2020. +* [Talks on Saint HighLoad++ in St. Petersburg](https://www.highload.ru/spb/2020/abstracts/6647) on April 6, 2020. From 1d083ed8127bb7a2a2f2c13462943f2d4099fbc0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Mar 2020 13:09:16 +0300 Subject: [PATCH 10/22] Added zeros test. --- .../queries/0_stateless/01096_zeros.reference | 48 +++++++++++++++++++ .../tests/queries/0_stateless/01096_zeros.sql | 12 +++++ 2 files changed, 60 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01096_zeros.reference create mode 100644 dbms/tests/queries/0_stateless/01096_zeros.sql diff --git a/dbms/tests/queries/0_stateless/01096_zeros.reference b/dbms/tests/queries/0_stateless/01096_zeros.reference new file mode 100644 index 00000000000..83e503406bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01096_zeros.reference @@ -0,0 +1,48 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +- +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +- +0 1000000000 +0 1000000000 +0 1000000000 +0 1000000000 diff --git a/dbms/tests/queries/0_stateless/01096_zeros.sql b/dbms/tests/queries/0_stateless/01096_zeros.sql new file mode 100644 index 00000000000..a6552b9229b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01096_zeros.sql @@ -0,0 +1,12 @@ +select zero from system.zeros limit 10 settings max_block_size = 3; +select '-'; +select zero from system.zeros_mt limit 10 settings max_block_size = 3, max_threads = 2; +select '-'; +select zero from zeros(10) settings max_block_size = 3; +select '-'; +select zero from zeros_mt(10) settings max_block_size = 3, max_threads=3; +select '-'; +select sum(zero), count() from (select * from system.zeros limit 1000000000); +select sum(zero), count() from (select * from system.zeros_mt limit 1000000000); +select sum(zero), count() from zeros(1000000000); +select sum(zero), count() from zeros_mt(1000000000); From 03259fc60ae4b04f7a3aec1784030858faf3fc85 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Mar 2020 13:28:24 +0300 Subject: [PATCH 11/22] Trying to improve tests --- .../queries/0_stateless/00834_kill_mutation.sh | 2 ++ ...34_kill_mutation_replicated_zookeeper.reference | 1 + .../00834_kill_mutation_replicated_zookeeper.sh | 14 ++++++++++++-- 3 files changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh index ed04c362401..5b6fc769d6a 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh @@ -59,6 +59,8 @@ ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = wait ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test.kill_mutation" +# must be empty +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.mutations WHERE table = 'kill_mutation' AND database = 'test' AND is_done = 0" ${CLICKHOUSE_CLIENT} --query="DROP TABLE test.kill_mutation" diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference index 9f1aa8e599d..3fe9a065099 100644 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.reference @@ -2,6 +2,7 @@ 0000000000 1 1 Code: 6, waiting test kill_mutation_r1 0000000000 Mutation 0000000000 was killed +0 *** Create and kill invalid mutation that blocks another mutation *** 0000000001 1 1 Code: 6, waiting test kill_mutation_r1 0000000001 diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh index 19c3fc4eed3..ec7c48147c4 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation_replicated_zookeeper.sh @@ -36,12 +36,17 @@ ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = wait -${CLICKHOUSE_CLIENT} --query="SELECT mutation_id FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" - +# No active mutations exists +${CLICKHOUSE_CLIENT} --query="SELECT count() FROM system.mutations WHERE database = 'test' AND table = 'kill_mutation_r1'" ${CLICKHOUSE_CLIENT} --query="SELECT '*** Create and kill invalid mutation that blocks another mutation ***'" ${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r1" +${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r2" + +# Should be empty, but in case of problems we will see some diagnostics +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.replication_queue WHERE table like 'kill_mutation_r%'" + ${CLICKHOUSE_CLIENT} --query="ALTER TABLE test.kill_mutation_r1 DELETE WHERE toUInt32(s) = 1" # good mutation, but blocked with wrong mutation @@ -63,7 +68,12 @@ ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = wait +${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r1" +${CLICKHOUSE_CLIENT} --query="SYSTEM SYNC REPLICA test.kill_mutation_r2" + ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test.kill_mutation_r2" +# must be empty +${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.mutations WHERE table = 'kill_mutation' AND database = 'test' AND is_done = 0" ${CLICKHOUSE_CLIENT} --query="DROP TABLE test.kill_mutation_r1" From ab2f2651f74ee62d76e43237c1fe979113192999 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Mar 2020 14:22:10 +0300 Subject: [PATCH 12/22] Update perftest after zeros. --- .../performance/complex_array_creation.xml | 4 +- dbms/tests/performance/conditional.xml | 16 +- .../performance/cryptographic_hashes.xml | 4 +- .../empty_string_deserialization.xml | 2 +- .../empty_string_serialization.xml | 2 +- dbms/tests/performance/float_formatting.xml | 24 +- dbms/tests/performance/float_parsing.xml | 22 +- dbms/tests/performance/functions_coding.xml | 4 +- dbms/tests/performance/functions_geo.xml | 4 +- .../performance/general_purpose_hashes.xml | 4 +- .../general_purpose_hashes_on_UUID.xml | 4 +- dbms/tests/performance/great_circle_dist.xml | 2 +- dbms/tests/performance/h3.xml | 2 +- dbms/tests/performance/if_array_num.xml | 12 +- dbms/tests/performance/if_array_string.xml | 12 +- dbms/tests/performance/if_string_const.xml | 8 +- .../performance/json_extract_rapidjson.xml | 26 +- .../performance/json_extract_simdjson.xml | 26 +- .../performance/logical_functions_large.xml | 20 +- .../performance/logical_functions_medium.xml | 40 +- .../performance/logical_functions_small.xml | 28 +- dbms/tests/performance/merge_tree_huge_pk.xml | 2 +- dbms/tests/performance/out.txt | 141 ++++++ dbms/tests/performance/out2.txt | 141 ++++++ .../performance/random_printable_ascii.xml | 14 +- dbms/tests/performance/res.txt | 465 ++++++++++++++++++ dbms/tests/performance/res2.txt | 465 ++++++++++++++++++ dbms/tests/performance/round_down.xml | 8 +- dbms/tests/performance/set.xml | 4 +- dbms/tests/performance/set_index.xml | 2 +- dbms/tests/performance/sort.xml | 20 +- dbms/tests/performance/string_join.xml | 2 +- dbms/tests/performance/string_set.xml | 2 +- .../synthetic_hardware_benchmark.xml | 6 +- .../vectorize_aggregation_combinators.xml | 2 +- .../performance/visit_param_extract_raw.xml | 2 +- 36 files changed, 1383 insertions(+), 159 deletions(-) create mode 100644 dbms/tests/performance/out.txt create mode 100644 dbms/tests/performance/out2.txt create mode 100644 dbms/tests/performance/res.txt create mode 100644 dbms/tests/performance/res2.txt diff --git a/dbms/tests/performance/complex_array_creation.xml b/dbms/tests/performance/complex_array_creation.xml index abcea2671e7..53f29d5b2b0 100644 --- a/dbms/tests/performance/complex_array_creation.xml +++ b/dbms/tests/performance/complex_array_creation.xml @@ -7,6 +7,6 @@ - SELECT count() FROM numbers(1000000) WHERE NOT ignore([[number], [number]]) - SELECT count() FROM numbers(1000000) WHERE NOT ignore([[], [number]]) + SELECT count() FROM zeros(1000000) WHERE NOT ignore([[zero], [zero]]) + SELECT count() FROM zeros(1000000) WHERE NOT ignore([[], [zero]]) diff --git a/dbms/tests/performance/conditional.xml b/dbms/tests/performance/conditional.xml index 2a4f4ccad6e..7449c57587d 100644 --- a/dbms/tests/performance/conditional.xml +++ b/dbms/tests/performance/conditional.xml @@ -6,13 +6,13 @@ - SELECT count() FROM numbers(1000000) WHERE NOT ignore(if(rand() % 2, toDateTime('2019-02-04 01:24:31'), toDate('2019-02-04'))) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(multiIf(rand() % 2, toDateTime('2019-02-04 01:24:31'), toDate('2019-02-04'))) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(if(rand() % 2, [toDateTime('2019-02-04 01:24:31')], [toDate('2019-02-04')])) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(multiIf(rand() % 2, [toDateTime('2019-02-04 01:24:31')], [toDate('2019-02-04')])) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(if(rand() % 2, toDateTime('2019-02-04 01:24:31'), toDate('2019-02-04'))) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(multiIf(rand() % 2, toDateTime('2019-02-04 01:24:31'), toDate('2019-02-04'))) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(if(rand() % 2, [toDateTime('2019-02-04 01:24:31')], [toDate('2019-02-04')])) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(multiIf(rand() % 2, [toDateTime('2019-02-04 01:24:31')], [toDate('2019-02-04')])) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(if(rand() % 2, toDateTime(rand()), toDate(rand()))) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(multiIf(rand() % 2, toDateTime(rand()), toDate(rand()))) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(if(rand() % 2, [toDateTime(rand())], [toDate(rand())])) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(multiIf(rand() % 2, [toDateTime(rand())], [toDate(rand())])) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(if(rand() % 2, toDateTime(rand()), toDate(rand()))) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(multiIf(rand() % 2, toDateTime(rand()), toDate(rand()))) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(if(rand() % 2, [toDateTime(rand())], [toDate(rand())])) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(multiIf(rand() % 2, [toDateTime(rand())], [toDate(rand())])) diff --git a/dbms/tests/performance/cryptographic_hashes.xml b/dbms/tests/performance/cryptographic_hashes.xml index 6dcc05d17fb..71d314ebf7f 100644 --- a/dbms/tests/performance/cryptographic_hashes.xml +++ b/dbms/tests/performance/cryptographic_hashes.xml @@ -46,8 +46,8 @@ table_slow - numbers(100000) - numbers_mt(1000000) + zeros(100000) + zeros_mt(1000000) diff --git a/dbms/tests/performance/empty_string_deserialization.xml b/dbms/tests/performance/empty_string_deserialization.xml index d4b4f338a85..5181c31d057 100644 --- a/dbms/tests/performance/empty_string_deserialization.xml +++ b/dbms/tests/performance/empty_string_deserialization.xml @@ -11,7 +11,7 @@ --> CREATE TABLE empty_strings (s String) ENGINE = Log; - INSERT INTO empty_strings SELECT '' FROM numbers_mt(100000000); + INSERT INTO empty_strings SELECT '' FROM zeros_mt(100000000); SELECT count() FROM empty_strings diff --git a/dbms/tests/performance/empty_string_serialization.xml b/dbms/tests/performance/empty_string_serialization.xml index 62b2e13f0af..46b76eaea72 100644 --- a/dbms/tests/performance/empty_string_serialization.xml +++ b/dbms/tests/performance/empty_string_serialization.xml @@ -11,6 +11,6 @@ --> CREATE TABLE empty_strings (s String) ENGINE = Log; - INSERT INTO empty_strings SELECT '' FROM numbers(100000000); + INSERT INTO empty_strings SELECT '' FROM zeros(100000000); DROP TABLE IF EXISTS empty_strings diff --git a/dbms/tests/performance/float_formatting.xml b/dbms/tests/performance/float_formatting.xml index 941e1bcaaa8..d3066bbd6a7 100644 --- a/dbms/tests/performance/float_formatting.xml +++ b/dbms/tests/performance/float_formatting.xml @@ -17,9 +17,6 @@ expr - 1 / rand() - rand() / 0xFFFFFFFF - 0xFFFFFFFF / rand() toFloat64(number) toFloat64(number % 2) toFloat64(number % 10) @@ -33,8 +30,6 @@ number / 3 number / 7 number / 16 - toFloat64(rand()) - toFloat64(rand64()) toFloat32(number) toFloat32(number % 2) toFloat32(number % 10) @@ -44,13 +39,24 @@ toFloat32(number % 100 + 0.5) toFloat32(number % 100 + 0.123) toFloat32(number % 1000 + 0.123456) - toFloat32(rand()) - toFloat32(rand64()) - reinterpretAsFloat32(reinterpretAsString(rand())) - reinterpretAsFloat64(reinterpretAsString(rand64())) + + expr_zero + + 1 / rand() + rand() / 0xFFFFFFFF + 0xFFFFFFFF / rand() + toFloat64(rand()) + toFloat64(rand64()) + toFloat32(rand()) + toFloat32(rand64()) + reinterpretAsFloat32(reinterpretAsString(rand())) + reinterpretAsFloat64(reinterpretAsString(rand64())) + + SELECT count() FROM numbers(1000000) WHERE NOT ignore(toString({expr})) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(toString({expr_zero})) diff --git a/dbms/tests/performance/float_parsing.xml b/dbms/tests/performance/float_parsing.xml index f75de81c698..f7a0ea62245 100644 --- a/dbms/tests/performance/float_parsing.xml +++ b/dbms/tests/performance/float_parsing.xml @@ -17,19 +17,25 @@ expr - toString(1 / rand()) - toString(rand() / 0xFFFFFFFF) - toString(0xFFFFFFFF / rand()) toString(number) toString(number % 10) - toString(rand()) - toString(rand64()) - concat(toString(rand(1)), '.', toString(rand(2))) - concat(toString(rand(1)), 'e', toString(rand(2) % 100)) - concat(toString(rand64(1)), toString(rand64(2)), toString(rand64(3))) + + expr_zero + + toString(1 / rand()) + toString(rand() / 0xFFFFFFFF) + toString(0xFFFFFFFF / rand()) + toString(rand()) + toString(rand64()) + concat(toString(rand(1)), '.', toString(rand(2))) + concat(toString(rand(1)), 'e', toString(rand(2) % 100)) + concat(toString(rand64(1)), toString(rand64(2)), toString(rand64(3))) + + SELECT count() FROM numbers(1000000) WHERE NOT ignore(toFloat64({expr})) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(toFloat64({expr_zero})) diff --git a/dbms/tests/performance/functions_coding.xml b/dbms/tests/performance/functions_coding.xml index 52ada24b5bc..a83ffbbab86 100644 --- a/dbms/tests/performance/functions_coding.xml +++ b/dbms/tests/performance/functions_coding.xml @@ -8,6 +8,6 @@ SELECT count() FROM numbers(1000000) WHERE NOT ignore(MACNumToString(number)) SELECT count() FROM numbers(1000000) WHERE NOT ignore(MACStringToNum(MACNumToString(number))) - SELECT count() FROM numbers_mt(10000000) WHERE NOT ignore(MACNumToString(rand64())) - SELECT count() FROM numbers_mt(10000000) WHERE NOT ignore(MACStringToNum(MACNumToString(rand64()))) + SELECT count() FROM zeros_mt(10000000) WHERE NOT ignore(MACNumToString(rand64())) + SELECT count() FROM zeros_mt(10000000) WHERE NOT ignore(MACStringToNum(MACNumToString(rand64()))) diff --git a/dbms/tests/performance/functions_geo.xml b/dbms/tests/performance/functions_geo.xml index 207d39c52b7..a3d2282e4e2 100644 --- a/dbms/tests/performance/functions_geo.xml +++ b/dbms/tests/performance/functions_geo.xml @@ -10,8 +10,8 @@ SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashEncode((number % 150)*1.1 - 75, (number * 3.14 % 300)*1.1 - 150)) SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashDecode(toString(number % 1000000))) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashEncode(1.0/rand(), 2.0/rand())) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashDecode(toString(rand() % 1000000))) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(geohashEncode(1.0/rand(), 2.0/rand())) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(geohashDecode(toString(rand() % 1000000))) SELECT count() FROM numbers(1000000) WHERE NOT ignore(geohashEncode(number + 91.0, number + 181.0)) diff --git a/dbms/tests/performance/general_purpose_hashes.xml b/dbms/tests/performance/general_purpose_hashes.xml index cc40c7fe1e3..4746271d664 100644 --- a/dbms/tests/performance/general_purpose_hashes.xml +++ b/dbms/tests/performance/general_purpose_hashes.xml @@ -47,8 +47,8 @@ table_slow - numbers(100000) - numbers_mt(1000000) + zeros(100000) + zeros_mt(1000000) diff --git a/dbms/tests/performance/general_purpose_hashes_on_UUID.xml b/dbms/tests/performance/general_purpose_hashes_on_UUID.xml index 3cb14e4c87c..9b749ae79e0 100644 --- a/dbms/tests/performance/general_purpose_hashes_on_UUID.xml +++ b/dbms/tests/performance/general_purpose_hashes_on_UUID.xml @@ -45,8 +45,8 @@ table - numbers(100000000) - numbers_mt(1000000000) + zeros(100000000) + zeros_mt(1000000000) diff --git a/dbms/tests/performance/great_circle_dist.xml b/dbms/tests/performance/great_circle_dist.xml index 3b88d00eb63..3b2aac65230 100644 --- a/dbms/tests/performance/great_circle_dist.xml +++ b/dbms/tests/performance/great_circle_dist.xml @@ -9,6 +9,6 @@ SELECT count() FROM numbers(1000000) WHERE NOT ignore(greatCircleDistance((rand(1) % 360) * 1. - 180, (number % 150) * 1.2 - 90, (number % 360) + toFloat64(rand(2)) / 4294967296 - 180, (rand(3) % 180) * 1. - 90)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(greatCircleDistance(55. + toFloat64(rand(1)) / 4294967296, 37. + toFloat64(rand(2)) / 4294967296, 55. + toFloat64(rand(3)) / 4294967296, 37. + toFloat64(rand(4)) / 4294967296)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(greatCircleDistance(55. + toFloat64(rand(1)) / 4294967296, 37. + toFloat64(rand(2)) / 4294967296, 55. + toFloat64(rand(3)) / 4294967296, 37. + toFloat64(rand(4)) / 4294967296)) diff --git a/dbms/tests/performance/h3.xml b/dbms/tests/performance/h3.xml index a09ac88f727..104e777fcc5 100644 --- a/dbms/tests/performance/h3.xml +++ b/dbms/tests/performance/h3.xml @@ -8,5 +8,5 @@ - SELECT count() FROM numbers(100000) WHERE NOT ignore(geoToH3(37.62 + rand(1) / 0x100000000, 55.75 + rand(2) / 0x100000000, 15)) + SELECT count() FROM zeros(100000) WHERE NOT ignore(geoToH3(37.62 + rand(1) / 0x100000000, 55.75 + rand(2) / 0x100000000, 15)) diff --git a/dbms/tests/performance/if_array_num.xml b/dbms/tests/performance/if_array_num.xml index 4ae4db3afdc..4ecd1e66daa 100644 --- a/dbms/tests/performance/if_array_num.xml +++ b/dbms/tests/performance/if_array_num.xml @@ -7,10 +7,10 @@ - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : [4, 5]) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : materialize([4, 5])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize([1, 2, 3]) : materialize([4, 5])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : [400, 500]) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : materialize([400, 500])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize([1, 2, 3]) : materialize([400, 500])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : [4, 5]) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : materialize([4, 5])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize([1, 2, 3]) : materialize([4, 5])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : [400, 500]) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? [1, 2, 3] : materialize([400, 500])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize([1, 2, 3]) : materialize([400, 500])) diff --git a/dbms/tests/performance/if_array_string.xml b/dbms/tests/performance/if_array_string.xml index 95dfb809230..40302131665 100644 --- a/dbms/tests/performance/if_array_string.xml +++ b/dbms/tests/performance/if_array_string.xml @@ -7,10 +7,10 @@ - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : ['a', 'b', 'c']) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : ['a', 'b', 'c']) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : materialize(['a', 'b', 'c'])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : materialize(['a', 'b', 'c'])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['', '']) : emptyArrayString()) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['https://github.com/ClickHouse/ClickHouse/pull/1070', 'https://www.google.ru/search?newwindow=1&site=&source=hp&q=zookeeper+wire+protocol+exists&oq=zookeeper+wire+protocol+exists&gs_l=psy-ab.3...330.6300.0.6687.33.28.0.0.0.0.386.4838.0j5j9j5.19.0....0...1.1.64.psy-ab..14.17.4448.0..0j35i39k1j0i131k1j0i22i30k1j0i19k1j33i21k1.r_3uFoNOrSU']) : emptyArrayString()) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : ['a', 'b', 'c']) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : ['a', 'b', 'c']) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? ['Hello', 'World'] : materialize(['a', 'b', 'c'])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['Hello', 'World']) : materialize(['a', 'b', 'c'])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['', '']) : emptyArrayString()) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(rand() % 2 ? materialize(['https://github.com/ClickHouse/ClickHouse/pull/1070', 'https://www.google.ru/search?newwindow=1&site=&source=hp&q=zookeeper+wire+protocol+exists&oq=zookeeper+wire+protocol+exists&gs_l=psy-ab.3...330.6300.0.6687.33.28.0.0.0.0.386.4838.0j5j9j5.19.0....0...1.1.64.psy-ab..14.17.4448.0..0j35i39k1j0i131k1j0i22i30k1j0i19k1j33i21k1.r_3uFoNOrSU']) : emptyArrayString()) diff --git a/dbms/tests/performance/if_string_const.xml b/dbms/tests/performance/if_string_const.xml index 5ab8455c948..7e273db36d8 100644 --- a/dbms/tests/performance/if_string_const.xml +++ b/dbms/tests/performance/if_string_const.xml @@ -6,8 +6,8 @@ - SELECT count() FROM numbers(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : 'world') - SELECT count() FROM numbers(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : '') - SELECT count() FROM numbers(1000000) WHERE NOT ignore(rand() % 2 ? toFixedString('hello', 5) : toFixedString('world', 5)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(rand() % 2 ? '' : toFixedString('world', 5)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : 'world') + SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? 'hello' : '') + SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? toFixedString('hello', 5) : toFixedString('world', 5)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(rand() % 2 ? '' : toFixedString('world', 5)) diff --git a/dbms/tests/performance/json_extract_rapidjson.xml b/dbms/tests/performance/json_extract_rapidjson.xml index 42b89456c9c..9818abb8581 100644 --- a/dbms/tests/performance/json_extract_rapidjson.xml +++ b/dbms/tests/performance/json_extract_rapidjson.xml @@ -31,18 +31,18 @@ 0 - SELECT 'rapidjson-1', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam')) - SELECT 'rapidjson-2', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam', 'nested_1')) - SELECT 'rapidjson-3', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractInt(materialize({json}), 'nparam')) - SELECT 'rapidjson-4', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({json}), 'nparam')) - SELECT 'rapidjson-5', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({json}), 'fparam')) + SELECT 'rapidjson-1', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam')) + SELECT 'rapidjson-2', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam', 'nested_1')) + SELECT 'rapidjson-3', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractInt(materialize({json}), 'nparam')) + SELECT 'rapidjson-4', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({json}), 'nparam')) + SELECT 'rapidjson-5', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({json}), 'fparam')) - SELECT 'rapidjson-6', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam')) - SELECT 'rapidjson-7', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam', 'nested_1')) - SELECT 'rapidjson-8', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractInt(materialize({long_json}), 'nparam')) - SELECT 'rapidjson-9', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({long_json}), 'nparam')) - SELECT 'rapidjson-10', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractRaw(materialize({long_json}), 'fparam')) - SELECT 'rapidjson-11', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam')) - SELECT 'rapidjson-12', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam', 'nested_2', -2)) - SELECT 'rapidjson-13', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractBool(materialize({long_json}), 'bparam')) + SELECT 'rapidjson-6', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam')) + SELECT 'rapidjson-7', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam', 'nested_1')) + SELECT 'rapidjson-8', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractInt(materialize({long_json}), 'nparam')) + SELECT 'rapidjson-9', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({long_json}), 'nparam')) + SELECT 'rapidjson-10', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractRaw(materialize({long_json}), 'fparam')) + SELECT 'rapidjson-11', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam')) + SELECT 'rapidjson-12', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam', 'nested_2', -2)) + SELECT 'rapidjson-13', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractBool(materialize({long_json}), 'bparam')) diff --git a/dbms/tests/performance/json_extract_simdjson.xml b/dbms/tests/performance/json_extract_simdjson.xml index 1e0c992802e..fa18d43df3e 100644 --- a/dbms/tests/performance/json_extract_simdjson.xml +++ b/dbms/tests/performance/json_extract_simdjson.xml @@ -31,19 +31,19 @@ 1 - SELECT 'simdjson-1', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam')) - SELECT 'simdjson-2', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam', 'nested_1')) - SELECT 'simdjson-3', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractInt(materialize({json}), 'nparam')) - SELECT 'simdjson-4', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({json}), 'nparam')) - SELECT 'simdjson-5', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({json}), 'fparam')) + SELECT 'simdjson-1', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam')) + SELECT 'simdjson-2', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({json}), 'sparam', 'nested_1')) + SELECT 'simdjson-3', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractInt(materialize({json}), 'nparam')) + SELECT 'simdjson-4', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({json}), 'nparam')) + SELECT 'simdjson-5', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({json}), 'fparam')) - SELECT 'simdjson-6', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam')) - SELECT 'simdjson-7', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam', 'nested_1')) - SELECT 'simdjson-8', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractInt(materialize({long_json}), 'nparam')) - SELECT 'simdjson-9', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({long_json}), 'nparam')) - SELECT 'simdjson-10', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractRaw(materialize({long_json}), 'fparam')) - SELECT 'simdjson-11', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam')) - SELECT 'simdjson-12', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam', 'nested_2', -2)) - SELECT 'simdjson-13', count() FROM numbers(1000000) WHERE NOT ignore(JSONExtractBool(materialize({long_json}), 'bparam')) + SELECT 'simdjson-6', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam')) + SELECT 'simdjson-7', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractString(materialize({long_json}), 'sparam', 'nested_1')) + SELECT 'simdjson-8', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractInt(materialize({long_json}), 'nparam')) + SELECT 'simdjson-9', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractUInt(materialize({long_json}), 'nparam')) + SELECT 'simdjson-10', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractRaw(materialize({long_json}), 'fparam')) + SELECT 'simdjson-11', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam')) + SELECT 'simdjson-12', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractFloat(materialize({long_json}), 'fparam', 'nested_2', -2)) + SELECT 'simdjson-13', count() FROM zeros(1000000) WHERE NOT ignore(JSONExtractBool(materialize({long_json}), 'bparam')) diff --git a/dbms/tests/performance/logical_functions_large.xml b/dbms/tests/performance/logical_functions_large.xml index a87b41ec916..e199094c43c 100644 --- a/dbms/tests/performance/logical_functions_large.xml +++ b/dbms/tests/performance/logical_functions_large.xml @@ -17,34 +17,34 @@ SELECT count() FROM - (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM numbers(20000000)) + (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM zeros(20000000)) WHERE NOT ignore(and(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3, materialize(0) AS x4, materialize(0) AS x5, materialize(0) AS x6, materialize(0) AS x7, materialize(0) AS x8, materialize(0) AS x9, materialize(0) AS x10 FROM numbers(150000000)) + (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3, materialize(0) AS x4, materialize(0) AS x5, materialize(0) AS x6, materialize(0) AS x7, materialize(0) AS x8, materialize(0) AS x9, materialize(0) AS x10 FROM zeros(150000000)) WHERE NOT ignore(and(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3, materialize(0) AS x4, materialize(0) AS x5, materialize(0) AS x6, materialize(0) AS x7, materialize(0) AS x8, materialize(0) AS x9, materialize(0) AS x10 FROM numbers(20000000)) + (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3, materialize(0) AS x4, materialize(0) AS x5, materialize(0) AS x6, materialize(0) AS x7, materialize(0) AS x8, materialize(0) AS x9, materialize(0) AS x10 FROM zeros(20000000)) WHERE NOT ignore(or(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM numbers(100000000)) + (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM numbers(500000000)) + (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3, materialize(1) AS x4, materialize(1) AS x5, materialize(1) AS x6, materialize(1) AS x7, materialize(1) AS x8, materialize(1) AS x9, materialize(1) AS x10 FROM zeros(500000000)) WHERE NOT ignore(xor(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM numbers(20000000)) + (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM zeros(20000000)) WHERE NOT ignore(and(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(toUInt8(0)) AS x1, materialize(toUInt16(0)) AS x2, materialize(toUInt32(0)) AS x3, materialize(toUInt64(0)) AS x4, materialize(toInt8(0)) AS x5, materialize(toInt16(0)) AS x6, materialize(toInt32(0)) AS x7, materialize(toInt64(0)) AS x8, materialize(toFloat32(0)) AS x9, materialize(toFloat64(0)) AS x10 FROM numbers(40000000)) + (SELECT materialize(toUInt8(0)) AS x1, materialize(toUInt16(0)) AS x2, materialize(toUInt32(0)) AS x3, materialize(toUInt64(0)) AS x4, materialize(toInt8(0)) AS x5, materialize(toInt16(0)) AS x6, materialize(toInt32(0)) AS x7, materialize(toInt64(0)) AS x8, materialize(toFloat32(0)) AS x9, materialize(toFloat64(0)) AS x10 FROM zeros(40000000)) WHERE NOT ignore(and(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(toUInt8(0)) AS x1, materialize(toUInt16(0)) AS x2, materialize(toUInt32(0)) AS x3, materialize(toUInt64(0)) AS x4, materialize(toInt8(0)) AS x5, materialize(toInt16(0)) AS x6, materialize(toInt32(0)) AS x7, materialize(toInt64(0)) AS x8, materialize(toFloat32(0)) AS x9, materialize(toFloat64(0)) AS x10 FROM numbers(20000000)) + (SELECT materialize(toUInt8(0)) AS x1, materialize(toUInt16(0)) AS x2, materialize(toUInt32(0)) AS x3, materialize(toUInt64(0)) AS x4, materialize(toInt8(0)) AS x5, materialize(toInt16(0)) AS x6, materialize(toInt32(0)) AS x7, materialize(toInt64(0)) AS x8, materialize(toFloat32(0)) AS x9, materialize(toFloat64(0)) AS x10 FROM zeros(20000000)) WHERE NOT ignore(or(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM numbers(20000000)) + (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM zeros(20000000)) WHERE NOT ignore(or(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) SELECT count() FROM - (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM numbers(20000000)) + (SELECT materialize(toUInt8(1)) AS x1, materialize(toUInt16(1)) AS x2, materialize(toUInt32(1)) AS x3, materialize(toUInt64(1)) AS x4, materialize(toInt8(1)) AS x5, materialize(toInt16(1)) AS x6, materialize(toInt32(1)) AS x7, materialize(toInt64(1)) AS x8, materialize(toFloat32(1)) AS x9, materialize(toFloat64(1)) AS x10 FROM zeros(20000000)) WHERE NOT ignore(xor(x1,x2,x3,x4,x5,x6,x7,x8,x9,x10)) diff --git a/dbms/tests/performance/logical_functions_medium.xml b/dbms/tests/performance/logical_functions_medium.xml index 087917040d9..1c4fd2a24dc 100644 --- a/dbms/tests/performance/logical_functions_medium.xml +++ b/dbms/tests/performance/logical_functions_medium.xml @@ -16,26 +16,26 @@ 1 - SELECT count() FROM (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3 FROM numbers(100000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3 FROM numbers(200000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3 FROM numbers(100000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3 FROM numbers(100000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3 FROM zeros(100000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3 FROM zeros(200000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(0) AS x1, materialize(0) AS x2, materialize(0) AS x3 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(1) AS x1, materialize(1) AS x2, materialize(1) AS x3 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(1) AS x1, materialize(toUInt32(1)) AS x2, materialize(toUInt64(1)) AS x3 FROM numbers(50000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(0) AS x1, materialize(toUInt32(0)) AS x2, materialize(toUInt64(0)) AS x3 FROM numbers(100000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(0) AS x1, materialize(toUInt32(0)) AS x2, materialize(toUInt64(0)) AS x3 FROM numbers(50000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(1) AS x1, materialize(toUInt32(1)) AS x2, materialize(toUInt64(1)) AS x3 FROM numbers(100000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(1) AS x1, materialize(toUInt32(1)) AS x2, materialize(toUInt64(1)) AS x3 FROM zeros(50000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(0) AS x1, materialize(toUInt32(0)) AS x2, materialize(toUInt64(0)) AS x3 FROM zeros(100000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(0) AS x1, materialize(toUInt32(0)) AS x2, materialize(toUInt64(0)) AS x3 FROM zeros(50000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(1) AS x1, materialize(toUInt32(1)) AS x2, materialize(toUInt64(1)) AS x3 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(1)) AS x1, materialize(toNullable(1)) AS x2, materialize(toNullable(1)) AS x3 FROM numbers(20000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(0)) AS x1, materialize(toNullable(0)) AS x2, materialize(toNullable(0)) AS x3 FROM numbers(50000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(0)) AS x1, materialize(toNullable(0)) AS x2, materialize(toNullable(0)) AS x3 FROM numbers(20000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(1)) AS x1, materialize(toNullable(1)) AS x2, materialize(toNullable(1)) AS x3 FROM numbers(50000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(1)) AS x1, materialize(toNullable(1)) AS x2, materialize(toNullable(1)) AS x3 FROM zeros(20000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(0)) AS x1, materialize(toNullable(0)) AS x2, materialize(toNullable(0)) AS x3 FROM zeros(50000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(0)) AS x1, materialize(toNullable(0)) AS x2, materialize(toNullable(0)) AS x3 FROM zeros(20000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(1)) AS x1, materialize(toNullable(1)) AS x2, materialize(toNullable(1)) AS x3 FROM zeros(50000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(toUInt16(1))) AS x1, materialize(toNullable(toUInt32(1))) AS x2, materialize(toNullable(toUInt64(1))) AS x3 FROM numbers(20000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(toUInt16(0))) AS x1, materialize(toNullable(toUInt32(0))) AS x2, materialize(toNullable(toUInt64(0))) AS x3 FROM numbers(50000000)) WHERE NOT ignore(and(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(toUInt16(0))) AS x1, materialize(toNullable(toUInt32(0))) AS x2, materialize(toNullable(toUInt64(0))) AS x3 FROM numbers(20000000)) WHERE NOT ignore(or(x1,x2,x3)) - SELECT count() FROM (SELECT materialize(toNullable(toUInt16(1))) AS x1, materialize(toNullable(toUInt32(1))) AS x2, materialize(toNullable(toUInt64(1))) AS x3 FROM numbers(50000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(toUInt16(1))) AS x1, materialize(toNullable(toUInt32(1))) AS x2, materialize(toNullable(toUInt64(1))) AS x3 FROM zeros(20000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(toUInt16(0))) AS x1, materialize(toNullable(toUInt32(0))) AS x2, materialize(toNullable(toUInt64(0))) AS x3 FROM zeros(50000000)) WHERE NOT ignore(and(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(toUInt16(0))) AS x1, materialize(toNullable(toUInt32(0))) AS x2, materialize(toNullable(toUInt64(0))) AS x3 FROM zeros(20000000)) WHERE NOT ignore(or(x1,x2,x3)) + SELECT count() FROM (SELECT materialize(toNullable(toUInt16(1))) AS x1, materialize(toNullable(toUInt32(1))) AS x2, materialize(toNullable(toUInt64(1))) AS x3 FROM zeros(50000000)) WHERE NOT ignore(or(x1,x2,x3)) CREATE DATABASE IF NOT EXISTS test_logical_functions @@ -44,10 +44,10 @@ CREATE TABLE test_logical_functions._4_1_Mixed (x1 UInt16, x2 UInt32, x3 UInt64, x4 UInt8) ENGINE = Memory CREATE TABLE test_logical_functions._4_1_Nullable_UInt8 (x1 Nullable(UInt8), x2 Nullable(UInt8), x3 Nullable(UInt8), x4 Nullable(UInt8)) ENGINE = Memory CREATE TABLE test_logical_functions._4_1_Nullable_Mixed (x1 Nullable(UInt16), x2 Nullable(UInt32), x3 Nullable(UInt64), x4 Nullable(Float64)) ENGINE = Memory - INSERT INTO test_logical_functions._4_1_UInt8 SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM numbers(500000000) - INSERT INTO test_logical_functions._4_1_Mixed SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM numbers(150000000) - INSERT INTO test_logical_functions._4_1_Nullable_UInt8 SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM numbers(150000000) - INSERT INTO test_logical_functions._4_1_Nullable_Mixed SELECT materialize(1), materialize(1), materialize(1), materialize(1.0) FROM numbers(50000000) + INSERT INTO test_logical_functions._4_1_UInt8 SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM zeros(500000000) + INSERT INTO test_logical_functions._4_1_Mixed SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM zeros(150000000) + INSERT INTO test_logical_functions._4_1_Nullable_UInt8 SELECT materialize(1), materialize(1), materialize(1), materialize(1) FROM zeros(150000000) + INSERT INTO test_logical_functions._4_1_Nullable_Mixed SELECT materialize(1), materialize(1), materialize(1), materialize(1.0) FROM zeros(50000000) SELECT count() FROM test_logical_functions._4_1_UInt8 WHERE NOT ignore(xor(x1,x2,x3,x4)) SELECT count() FROM test_logical_functions._4_1_Mixed WHERE NOT ignore(xor(x1,x2,x3,x4)) diff --git a/dbms/tests/performance/logical_functions_small.xml b/dbms/tests/performance/logical_functions_small.xml index ed6ab2afde6..d3d7a2eecca 100644 --- a/dbms/tests/performance/logical_functions_small.xml +++ b/dbms/tests/performance/logical_functions_small.xml @@ -25,8 +25,8 @@ CREATE TABLE test_logical_functions._2_0_UInt8 (x1 UInt8, x2 UInt8) ENGINE = Memory CREATE TABLE test_logical_functions._2_1_UInt8 (x1 UInt8, x2 UInt8) ENGINE = Memory - INSERT INTO test_logical_functions._2_0_UInt8 SELECT materialize(0) AS x1, materialize(0) AS x2 FROM numbers(1000000000) - INSERT INTO test_logical_functions._2_1_UInt8 SELECT materialize(1) AS x1, materialize(1) AS x2 FROM numbers(1000000000) + INSERT INTO test_logical_functions._2_0_UInt8 SELECT materialize(0) AS x1, materialize(0) AS x2 FROM zeros(1000000000) + INSERT INTO test_logical_functions._2_1_UInt8 SELECT materialize(1) AS x1, materialize(1) AS x2 FROM zeros(1000000000) SELECT count() FROM test_logical_functions._2_1_UInt8 WHERE NOT ignore(and(x1,x2)) SELECT count() FROM test_logical_functions._2_0_UInt8 WHERE NOT ignore(and(x1,x2)) @@ -34,20 +34,20 @@ SELECT count() FROM test_logical_functions._2_1_UInt8 WHERE NOT ignore(or(x1,x2)) SELECT count() FROM test_logical_functions._2_1_UInt8 WHERE NOT ignore(xor(x1,x2)) - SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM numbers(30000000)) WHERE NOT ignore(and(x1,x2)) - SELECT count() FROM (SELECT toNullable(materialize(0)) AS x1, toNullable(materialize(0)) AS x2 FROM numbers(40000000)) WHERE NOT ignore(and(x1,x2)) - SELECT count() FROM (SELECT toNullable(materialize(0)) AS x1, toNullable(materialize(0)) AS x2 FROM numbers(40000000)) WHERE NOT ignore(or(x1,x2)) - SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM numbers(30000000)) WHERE NOT ignore(or(x1,x2)) - SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM numbers(100000000)) WHERE NOT ignore(xor(x1,x2)) + SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM zeros(30000000)) WHERE NOT ignore(and(x1,x2)) + SELECT count() FROM (SELECT toNullable(materialize(0)) AS x1, toNullable(materialize(0)) AS x2 FROM zeros(40000000)) WHERE NOT ignore(and(x1,x2)) + SELECT count() FROM (SELECT toNullable(materialize(0)) AS x1, toNullable(materialize(0)) AS x2 FROM zeros(40000000)) WHERE NOT ignore(or(x1,x2)) + SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM zeros(30000000)) WHERE NOT ignore(or(x1,x2)) + SELECT count() FROM (SELECT toNullable(materialize(1)) AS x1, toNullable(materialize(1)) AS x2 FROM zeros(100000000)) WHERE NOT ignore(xor(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt16(1)) AS x1, materialize(toUInt32(1)) AS x2 FROM numbers(200000000)) WHERE NOT ignore(and(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt32(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM numbers(100000000)) WHERE NOT ignore(and(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(0.1) AS x2 FROM numbers(100000000)) WHERE NOT ignore(and(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt16(1)) AS x1, materialize(toUInt32(1)) AS x2 FROM numbers(200000000)) WHERE NOT ignore(or(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt32(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM numbers(100000000)) WHERE NOT ignore(or(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(0.1) AS x2 FROM numbers(100000000)) WHERE NOT ignore(or(x1,x2)) - SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM numbers(100000000)) WHERE NOT ignore(xor(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt16(1)) AS x1, materialize(toUInt32(1)) AS x2 FROM zeros(200000000)) WHERE NOT ignore(and(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt32(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM zeros(100000000)) WHERE NOT ignore(and(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(0.1) AS x2 FROM zeros(100000000)) WHERE NOT ignore(and(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt16(1)) AS x1, materialize(toUInt32(1)) AS x2 FROM zeros(200000000)) WHERE NOT ignore(or(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt32(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(0.1) AS x2 FROM zeros(100000000)) WHERE NOT ignore(or(x1,x2)) + SELECT count() FROM (SELECT materialize(toUInt64(1)) AS x1, materialize(toUInt64(1)) AS x2 FROM zeros(100000000)) WHERE NOT ignore(xor(x1,x2)) DROP TABLE test_logical_functions._2_0_UInt8 DROP TABLE test_logical_functions._2_1_UInt8 diff --git a/dbms/tests/performance/merge_tree_huge_pk.xml b/dbms/tests/performance/merge_tree_huge_pk.xml index 1636fd52e2d..78a6cf6308e 100644 --- a/dbms/tests/performance/merge_tree_huge_pk.xml +++ b/dbms/tests/performance/merge_tree_huge_pk.xml @@ -190,7 +190,7 @@ rand64(691) % 5 as c691, rand64(692) % 5 as c692, rand64(693) % 5 as c693, rand64(694) % 5 as c694, rand64(695) % 5 as c695, rand64(696) % 5 as c696, rand64(697) % 5 as c697, rand64(698) % 5 as c698, rand64(699) % 5 as c699, rand64(700) % 5 as c700, rand64(701) % 5 as c701 - FROM system.numbers + FROM system.zeros LIMIT 1048576 diff --git a/dbms/tests/performance/out.txt b/dbms/tests/performance/out.txt new file mode 100644 index 00000000000..4c6490ef837 --- /dev/null +++ b/dbms/tests/performance/out.txt @@ -0,0 +1,141 @@ +var queries = +[ + { + "comment": "", + "query": "SELECT count() FROM dt where not ignore(x)" + }, + { + "comment": "", + "query": "SELECT count() FROM dt64 where not ignore(x)" + }, + { + "comment": "", + "query": "SELECT max(x) FROM dt" + }, + { + "comment": "", + "query": "SELECT max(x) FROM dt64" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toString(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toString(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toStartOfDay(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(addDays(x, 1))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))" + }, + { + "comment": "", + "query": "SELECT sum(x = x) FROM dt" + }, + { + "comment": "", + "query": "SELECT sum(x = x) FROM dt64" + }, + { + "comment": "", + "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt" + }, + { + "comment": "", + "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64" + } +] ; +var results = +[ + { + "system": "ClickHouse", + "comments": "", + "version": "19.18.1", + "result": [ + [ + 0.039 + ], + [ + 0.073 + ], + [ + 0.043 + ], + [ + 0.215 + ], + [ + 0.272 + ], + [ + 0.504 + ], + [ + 0.056 + ], + [ + 0.16 + ], + [ + 0.058 + ], + [ + 0.164 + ], + [ + 0.046 + ], + [ + 0.146 + ], + [ + 0.117 + ], + [ + 0.295 + ], + [ + 0.047 + ], + [ + 0.085 + ], + [ + 0.376 + ], + [ + 0.734 + ] + ], + "time": "2019-12-11 15:23:16", + "data_size": 100000000 + } +] ; diff --git a/dbms/tests/performance/out2.txt b/dbms/tests/performance/out2.txt new file mode 100644 index 00000000000..b5f740ea502 --- /dev/null +++ b/dbms/tests/performance/out2.txt @@ -0,0 +1,141 @@ +var queries = +[ + { + "comment": "", + "query": "SELECT count() FROM dt where not ignore(x)" + }, + { + "comment": "", + "query": "SELECT count() FROM dt64 where not ignore(x)" + }, + { + "comment": "", + "query": "SELECT max(x) FROM dt" + }, + { + "comment": "", + "query": "SELECT max(x) FROM dt64" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toString(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toString(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toStartOfDay(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))" + }, + { + "comment": "", + "query": "SELECT count() from dt where not ignore(addDays(x, 1))" + }, + { + "comment": "", + "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))" + }, + { + "comment": "", + "query": "SELECT sum(x = x) FROM dt" + }, + { + "comment": "", + "query": "SELECT sum(x = x) FROM dt64" + }, + { + "comment": "", + "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt" + }, + { + "comment": "", + "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64" + } +] ; +var results = +[ + { + "system": "ClickHouse", + "comments": "", + "version": "19.18.1", + "result": [ + [ + 0.039 + ], + [ + 0.074 + ], + [ + 0.044 + ], + [ + 0.222 + ], + [ + 0.281 + ], + [ + 0.529 + ], + [ + 0.06 + ], + [ + 0.178 + ], + [ + 0.06 + ], + [ + 0.173 + ], + [ + 0.048 + ], + [ + 0.146 + ], + [ + 0.117 + ], + [ + 0.296 + ], + [ + 0.048 + ], + [ + 0.086 + ], + [ + 0.378 + ], + [ + 0.737 + ] + ], + "time": "2019-12-11 16:23:52", + "data_size": 100000000 + } +] ; diff --git a/dbms/tests/performance/random_printable_ascii.xml b/dbms/tests/performance/random_printable_ascii.xml index 320ffeac796..29d7d645d18 100644 --- a/dbms/tests/performance/random_printable_ascii.xml +++ b/dbms/tests/performance/random_printable_ascii.xml @@ -7,11 +7,11 @@ - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(10)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(100)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(1000)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(10000)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 10)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 100)) - SELECT count() FROM numbers(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 1000)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(10)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(100)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(1000)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(10000)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 10)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 100)) + SELECT count() FROM zeros(1000000) WHERE NOT ignore(randomPrintableASCII(rand() % 1000)) diff --git a/dbms/tests/performance/res.txt b/dbms/tests/performance/res.txt new file mode 100644 index 00000000000..f3e04b06347 --- /dev/null +++ b/dbms/tests/performance/res.txt @@ -0,0 +1,465 @@ +[ +{ + "hostname": "nikkochetovdev.sas.yp-c.yandex.net", + "main_metric": "min_time", + "num_cores": 32, + "num_threads": 32, + "path": "/home/nik-kochetov/dev/c2/enmk/ClickHouse/dbms/tests/performance/date_time_64.xml", + "ram": 47475228672, + "runs": [ + { + "bytes_per_second": 9409861540.352007, + "memory_usage": 0, + "min_time": 0.039000, + "quantiles": { + "0.1": 0.040199, + "0.2": 0.040653, + "0.3": 0.041213, + "0.4": 0.041642, + "0.5": 0.042083, + "0.6": 0.042603, + "0.7": 0.043306, + "0.8": 0.043760, + "0.9": 0.044983, + "0.95": 0.046143, + "0.99": 0.047486, + "0.999": 0.058256, + "0.9999": 0.059333 + }, + "queries_per_second": 23.524654, + "query": "SELECT count() FROM dt where not ignore(x)", + "query_index": 0, + "rows_per_second": 2352465385.088002, + "total_time": 4.250860 + }, + { + "bytes_per_second": 10384233889.130009, + "memory_usage": 0, + "min_time": 0.073000, + "quantiles": { + "0.1": 0.075317, + "0.2": 0.075790, + "0.3": 0.076077, + "0.4": 0.076334, + "0.5": 0.076805, + "0.6": 0.077249, + "0.7": 0.077603, + "0.8": 0.078142, + "0.9": 0.079281, + "0.95": 0.079756, + "0.99": 0.081960, + "0.999": 0.083226, + "0.9999": 0.083353 + }, + "queries_per_second": 12.980292, + "query": "SELECT count() FROM dt64 where not ignore(x)", + "query_index": 1, + "rows_per_second": 1298029236.141251, + "total_time": 7.703987 + }, + { + "bytes_per_second": 8750237976.940920, + "memory_usage": 0, + "min_time": 0.043000, + "quantiles": { + "0.1": 0.044337, + "0.2": 0.044870, + "0.3": 0.045090, + "0.4": 0.045218, + "0.5": 0.045366, + "0.6": 0.045701, + "0.7": 0.046101, + "0.8": 0.046507, + "0.9": 0.047493, + "0.95": 0.048040, + "0.99": 0.049223, + "0.999": 0.050705, + "0.9999": 0.050854 + }, + "queries_per_second": 21.875595, + "query": "SELECT max(x) FROM dt", + "query_index": 2, + "rows_per_second": 2187559494.235230, + "total_time": 4.571304 + }, + { + "bytes_per_second": 3487337552.818883, + "memory_usage": 0, + "min_time": 0.215000, + "quantiles": { + "0.1": 0.221970, + "0.2": 0.223491, + "0.3": 0.225555, + "0.4": 0.226886, + "0.5": 0.229043, + "0.6": 0.229805, + "0.7": 0.230861, + "0.8": 0.233639, + "0.9": 0.236776, + "0.95": 0.241592, + "0.99": 0.253925, + "0.999": 0.254084, + "0.9999": 0.254100 + }, + "queries_per_second": 4.303441, + "query": "SELECT max(x) FROM dt64", + "query_index": 3, + "rows_per_second": 435917194.102360, + "total_time": 16.498425 + }, + { + "bytes_per_second": 1368202556.460040, + "memory_usage": 0, + "min_time": 0.272000, + "quantiles": { + "0.1": 0.279798, + "0.2": 0.283832, + "0.3": 0.284502, + "0.4": 0.286428, + "0.5": 0.288811, + "0.6": 0.291098, + "0.7": 0.296558, + "0.8": 0.298619, + "0.9": 0.304383, + "0.95": 0.306885, + "0.99": 0.312714, + "0.999": 0.317105, + "0.9999": 0.317544 + }, + "queries_per_second": 3.371624, + "query": "SELECT count() from dt where not ignore(toString(x))", + "query_index": 4, + "rows_per_second": 342050639.115010, + "total_time": 14.533056 + }, + { + "bytes_per_second": 1450828196.929232, + "memory_usage": 0, + "min_time": 0.504000, + "quantiles": { + "0.1": 0.513239, + "0.2": 0.528871, + "0.3": 0.532407, + "0.4": 0.538834, + "0.5": 0.542961, + "0.6": 0.545868, + "0.7": 0.547043, + "0.8": 0.554186, + "0.9": 0.568202, + "0.95": 0.583093, + "0.99": 0.648873, + "0.999": 0.663674, + "0.9999": 0.665154 + }, + "queries_per_second": 1.776884, + "query": "SELECT count() from dt64 where not ignore(toString(x))", + "query_index": 5, + "rows_per_second": 181353524.616154, + "total_time": 10.692875 + }, + { + "bytes_per_second": 6487544575.900535, + "memory_usage": 0, + "min_time": 0.056000, + "quantiles": { + "0.1": 0.058540, + "0.2": 0.059734, + "0.3": 0.060278, + "0.4": 0.060673, + "0.5": 0.061352, + "0.6": 0.062073, + "0.7": 0.062844, + "0.8": 0.063546, + "0.9": 0.064830, + "0.95": 0.066672, + "0.99": 0.067562, + "0.999": 0.068277, + "0.9999": 0.068349 + }, + "queries_per_second": 16.218861, + "query": "SELECT count() from dt where not ignore(toStartOfDay(x))", + "query_index": 6, + "rows_per_second": 1621886143.975134, + "total_time": 6.165661 + }, + { + "bytes_per_second": 4433432853.411924, + "memory_usage": 0, + "min_time": 0.160000, + "quantiles": { + "0.1": 0.168755, + "0.2": 0.174096, + "0.3": 0.175198, + "0.4": 0.177784, + "0.5": 0.179172, + "0.6": 0.180600, + "0.7": 0.183102, + "0.8": 0.185960, + "0.9": 0.189487, + "0.95": 0.190507, + "0.99": 0.193747, + "0.999": 0.195806, + "0.9999": 0.196012 + }, + "queries_per_second": 5.495579, + "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))", + "query_index": 7, + "rows_per_second": 554179106.676491, + "total_time": 12.919475 + }, + { + "bytes_per_second": 6450969380.532378, + "memory_usage": 0, + "min_time": 0.058000, + "quantiles": { + "0.1": 0.059490, + "0.2": 0.060198, + "0.3": 0.060511, + "0.4": 0.060950, + "0.5": 0.061638, + "0.6": 0.062165, + "0.7": 0.063180, + "0.8": 0.063964, + "0.9": 0.064843, + "0.95": 0.065833, + "0.99": 0.067410, + "0.999": 0.067703, + "0.9999": 0.067732 + }, + "queries_per_second": 16.127423, + "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))", + "query_index": 8, + "rows_per_second": 1612742345.133095, + "total_time": 6.200618 + }, + { + "bytes_per_second": 4347265724.547048, + "memory_usage": 0, + "min_time": 0.164000, + "quantiles": { + "0.1": 0.169796, + "0.2": 0.171982, + "0.3": 0.173706, + "0.4": 0.175892, + "0.5": 0.177192, + "0.6": 0.178714, + "0.7": 0.182275, + "0.8": 0.187131, + "0.9": 0.210764, + "0.95": 0.217970, + "0.99": 0.222446, + "0.999": 0.222648, + "0.9999": 0.222668 + }, + "queries_per_second": 5.391733, + "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))", + "query_index": 9, + "rows_per_second": 543408215.568381, + "total_time": 12.240961 + }, + { + "bytes_per_second": 8056446421.959373, + "memory_usage": 0, + "min_time": 0.046000, + "quantiles": { + "0.1": 0.047554, + "0.2": 0.048000, + "0.3": 0.048242, + "0.4": 0.048407, + "0.5": 0.048770, + "0.6": 0.049179, + "0.7": 0.049716, + "0.8": 0.050657, + "0.9": 0.053740, + "0.95": 0.054697, + "0.99": 0.057408, + "0.999": 0.059362, + "0.9999": 0.059557 + }, + "queries_per_second": 20.141116, + "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))", + "query_index": 10, + "rows_per_second": 2014111605.489843, + "total_time": 4.964968 + }, + { + "bytes_per_second": 5111454198.308989, + "memory_usage": 0, + "min_time": 0.146000, + "quantiles": { + "0.1": 0.150252, + "0.2": 0.151915, + "0.3": 0.153284, + "0.4": 0.154656, + "0.5": 0.155617, + "0.6": 0.156039, + "0.7": 0.158277, + "0.8": 0.159162, + "0.9": 0.161681, + "0.95": 0.164258, + "0.99": 0.169050, + "0.999": 0.172889, + "0.9999": 0.173273 + }, + "queries_per_second": 6.324464, + "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))", + "query_index": 11, + "rows_per_second": 638931774.788624, + "total_time": 10.751900 + }, + { + "bytes_per_second": 3227703999.626698, + "memory_usage": 0, + "min_time": 0.117000, + "quantiles": { + "0.1": 0.118657, + "0.2": 0.120602, + "0.3": 0.121634, + "0.4": 0.122749, + "0.5": 0.123620, + "0.6": 0.124714, + "0.7": 0.125342, + "0.8": 0.126662, + "0.9": 0.129363, + "0.95": 0.130856, + "0.99": 0.134388, + "0.999": 0.135462, + "0.9999": 0.135569 + }, + "queries_per_second": 7.983644, + "query": "SELECT count() from dt where not ignore(addDays(x, 1))", + "query_index": 12, + "rows_per_second": 806925999.906674, + "total_time": 10.772023 + }, + { + "bytes_per_second": 2564513384.816551, + "memory_usage": 0, + "min_time": 0.295000, + "quantiles": { + "0.1": 0.300345, + "0.2": 0.304122, + "0.3": 0.305735, + "0.4": 0.307827, + "0.5": 0.308609, + "0.6": 0.309365, + "0.7": 0.311653, + "0.8": 0.315120, + "0.9": 0.317984, + "0.95": 0.321220, + "0.99": 0.345205, + "0.999": 0.364580, + "0.9999": 0.366517 + }, + "queries_per_second": 3.162491, + "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))", + "query_index": 13, + "rows_per_second": 320564173.102069, + "total_time": 15.810323 + }, + { + "bytes_per_second": 7763363653.267178, + "memory_usage": 0, + "min_time": 0.047000, + "quantiles": { + "0.1": 0.049248, + "0.2": 0.050015, + "0.3": 0.050272, + "0.4": 0.050715, + "0.5": 0.051279, + "0.6": 0.051927, + "0.7": 0.052337, + "0.8": 0.052892, + "0.9": 0.053614, + "0.95": 0.055995, + "0.99": 0.057009, + "0.999": 0.059043, + "0.9999": 0.059246 + }, + "queries_per_second": 19.408409, + "query": "SELECT sum(x = x) FROM dt", + "query_index": 14, + "rows_per_second": 1940840913.316794, + "total_time": 5.152406 + }, + { + "bytes_per_second": 8536230820.533304, + "memory_usage": 0, + "min_time": 0.085000, + "quantiles": { + "0.1": 0.089260, + "0.2": 0.090209, + "0.3": 0.090926, + "0.4": 0.092020, + "0.5": 0.093539, + "0.6": 0.094379, + "0.7": 0.095554, + "0.8": 0.096637, + "0.9": 0.098793, + "0.95": 0.100480, + "0.99": 0.103172, + "0.999": 0.110254, + "0.9999": 0.110962 + }, + "queries_per_second": 10.670289, + "query": "SELECT sum(x = x) FROM dt64", + "query_index": 15, + "rows_per_second": 1067028852.566663, + "total_time": 9.371818 + }, + { + "bytes_per_second": 981290094.751588, + "memory_usage": 0, + "min_time": 0.376000, + "quantiles": { + "0.1": 0.390792, + "0.2": 0.392070, + "0.3": 0.395036, + "0.4": 0.398827, + "0.5": 0.404095, + "0.6": 0.408434, + "0.7": 0.412138, + "0.8": 0.417432, + "0.9": 0.423049, + "0.95": 0.437361, + "0.99": 0.455200, + "0.999": 0.458357, + "0.9999": 0.458672 + }, + "queries_per_second": 2.442035, + "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt", + "query_index": 16, + "rows_per_second": 245322523.687897, + "total_time": 25.388659 + }, + { + "bytes_per_second": 1028205501.170973, + "memory_usage": 0, + "min_time": 0.734000, + "quantiles": { + "0.1": 0.759821, + "0.2": 0.762149, + "0.3": 0.765029, + "0.4": 0.772696, + "0.5": 0.778148, + "0.6": 0.783112, + "0.7": 0.786080, + "0.8": 0.797778, + "0.9": 0.806538, + "0.95": 0.808824, + "0.99": 0.816510, + "0.999": 0.818605, + "0.9999": 0.818814 + }, + "queries_per_second": 1.236670, + "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64", + "query_index": 17, + "rows_per_second": 128525687.646372, + "total_time": 20.215586 + } + ], + "server_version": "19.18.1", + "test_name": "date_time_64", + "time": "2019-12-11 15:23:16" +} +] diff --git a/dbms/tests/performance/res2.txt b/dbms/tests/performance/res2.txt new file mode 100644 index 00000000000..217693cbdbd --- /dev/null +++ b/dbms/tests/performance/res2.txt @@ -0,0 +1,465 @@ +[ +{ + "hostname": "nikkochetovdev.sas.yp-c.yandex.net", + "main_metric": "min_time", + "num_cores": 32, + "num_threads": 32, + "path": "/home/nik-kochetov/dev/c2/enmk/ClickHouse/dbms/tests/performance/date_time_64.xml", + "ram": 47475228672, + "runs": [ + { + "bytes_per_second": 9512005376.525969, + "memory_usage": 0, + "min_time": 0.039000, + "quantiles": { + "0.1": 0.039925, + "0.2": 0.040346, + "0.3": 0.040838, + "0.4": 0.041114, + "0.5": 0.041425, + "0.6": 0.041956, + "0.7": 0.042440, + "0.8": 0.043235, + "0.9": 0.044221, + "0.95": 0.045529, + "0.99": 0.050977, + "0.999": 0.057833, + "0.9999": 0.058518 + }, + "queries_per_second": 23.780013, + "query": "SELECT count() FROM dt where not ignore(x)", + "query_index": 0, + "rows_per_second": 2378001344.131492, + "total_time": 4.205212 + }, + { + "bytes_per_second": 10059387738.161827, + "memory_usage": 0, + "min_time": 0.074000, + "quantiles": { + "0.1": 0.076402, + "0.2": 0.077200, + "0.3": 0.078096, + "0.4": 0.078585, + "0.5": 0.079304, + "0.6": 0.079890, + "0.7": 0.080571, + "0.8": 0.082080, + "0.9": 0.083230, + "0.95": 0.083941, + "0.99": 0.084464, + "0.999": 0.084568, + "0.9999": 0.084579 + }, + "queries_per_second": 12.574235, + "query": "SELECT count() FROM dt64 where not ignore(x)", + "query_index": 1, + "rows_per_second": 1257423467.270228, + "total_time": 7.952770 + }, + { + "bytes_per_second": 8551729880.329183, + "memory_usage": 0, + "min_time": 0.044000, + "quantiles": { + "0.1": 0.045402, + "0.2": 0.045667, + "0.3": 0.046041, + "0.4": 0.046223, + "0.5": 0.046410, + "0.6": 0.046785, + "0.7": 0.047063, + "0.8": 0.047522, + "0.9": 0.048746, + "0.95": 0.049961, + "0.99": 0.051003, + "0.999": 0.051011, + "0.9999": 0.051011 + }, + "queries_per_second": 21.379325, + "query": "SELECT max(x) FROM dt", + "query_index": 2, + "rows_per_second": 2137932470.082296, + "total_time": 4.677416 + }, + { + "bytes_per_second": 3302958415.675775, + "memory_usage": 0, + "min_time": 0.222000, + "quantiles": { + "0.1": 0.231731, + "0.2": 0.233341, + "0.3": 0.234829, + "0.4": 0.236819, + "0.5": 0.240804, + "0.6": 0.242230, + "0.7": 0.243392, + "0.8": 0.246855, + "0.9": 0.250680, + "0.95": 0.254090, + "0.99": 0.258972, + "0.999": 0.259751, + "0.9999": 0.259828 + }, + "queries_per_second": 4.094975, + "query": "SELECT max(x) FROM dt64", + "query_index": 3, + "rows_per_second": 412869801.959472, + "total_time": 12.942693 + }, + { + "bytes_per_second": 1274642546.723289, + "memory_usage": 0, + "min_time": 0.281000, + "quantiles": { + "0.1": 0.300769, + "0.2": 0.303915, + "0.3": 0.305376, + "0.4": 0.306813, + "0.5": 0.308576, + "0.6": 0.310090, + "0.7": 0.313469, + "0.8": 0.317862, + "0.9": 0.323054, + "0.95": 0.334075, + "0.99": 0.346969, + "0.999": 0.349500, + "0.9999": 0.349753 + }, + "queries_per_second": 3.153900, + "query": "SELECT count() from dt where not ignore(toString(x))", + "query_index": 4, + "rows_per_second": 318660636.680822, + "total_time": 10.780303 + }, + { + "bytes_per_second": 1428385007.182179, + "memory_usage": 0, + "min_time": 0.529000, + "quantiles": { + "0.1": 0.538657, + "0.2": 0.539444, + "0.3": 0.541329, + "0.4": 0.542470, + "0.5": 0.544262, + "0.6": 0.547909, + "0.7": 0.551176, + "0.8": 0.554685, + "0.9": 0.563354, + "0.95": 0.655805, + "0.99": 0.775895, + "0.999": 0.802141, + "0.9999": 0.804766 + }, + "queries_per_second": 1.722420, + "query": "SELECT count() from dt64 where not ignore(toString(x))", + "query_index": 5, + "rows_per_second": 178548125.897772, + "total_time": 15.675623 + }, + { + "bytes_per_second": 6202344999.343684, + "memory_usage": 0, + "min_time": 0.060000, + "quantiles": { + "0.1": 0.061787, + "0.2": 0.062539, + "0.3": 0.063240, + "0.4": 0.063643, + "0.5": 0.064330, + "0.6": 0.064915, + "0.7": 0.065499, + "0.8": 0.066099, + "0.9": 0.067622, + "0.95": 0.068367, + "0.99": 0.069822, + "0.999": 0.070812, + "0.9999": 0.070911 + }, + "queries_per_second": 15.505862, + "query": "SELECT count() from dt where not ignore(toStartOfDay(x))", + "query_index": 6, + "rows_per_second": 1550586249.835921, + "total_time": 6.449174 + }, + { + "bytes_per_second": 4216802240.766242, + "memory_usage": 0, + "min_time": 0.178000, + "quantiles": { + "0.1": 0.183190, + "0.2": 0.184556, + "0.3": 0.186327, + "0.4": 0.187608, + "0.5": 0.189150, + "0.6": 0.190982, + "0.7": 0.192435, + "0.8": 0.194643, + "0.9": 0.199023, + "0.95": 0.199987, + "0.99": 0.202437, + "0.999": 0.204918, + "0.9999": 0.205166 + }, + "queries_per_second": 5.217217, + "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))", + "query_index": 7, + "rows_per_second": 527100280.095780, + "total_time": 18.592288 + }, + { + "bytes_per_second": 6196564885.117176, + "memory_usage": 0, + "min_time": 0.060000, + "quantiles": { + "0.1": 0.062199, + "0.2": 0.062682, + "0.3": 0.063260, + "0.4": 0.064251, + "0.5": 0.064656, + "0.6": 0.065146, + "0.7": 0.065637, + "0.8": 0.066203, + "0.9": 0.066740, + "0.95": 0.067316, + "0.99": 0.068686, + "0.999": 0.069785, + "0.9999": 0.069895 + }, + "queries_per_second": 15.491412, + "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))", + "query_index": 8, + "rows_per_second": 1549141221.279294, + "total_time": 6.455189 + }, + { + "bytes_per_second": 4254486514.800510, + "memory_usage": 0, + "min_time": 0.173000, + "quantiles": { + "0.1": 0.177071, + "0.2": 0.179454, + "0.3": 0.181233, + "0.4": 0.182128, + "0.5": 0.183917, + "0.6": 0.185398, + "0.7": 0.189102, + "0.8": 0.194767, + "0.9": 0.199779, + "0.95": 0.202001, + "0.99": 0.215625, + "0.999": 0.223829, + "0.9999": 0.224649 + }, + "queries_per_second": 5.262679, + "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))", + "query_index": 9, + "rows_per_second": 531810814.350064, + "total_time": 10.640969 + }, + { + "bytes_per_second": 7876363631.561918, + "memory_usage": 0, + "min_time": 0.048000, + "quantiles": { + "0.1": 0.048766, + "0.2": 0.049063, + "0.3": 0.049456, + "0.4": 0.049926, + "0.5": 0.050485, + "0.6": 0.050856, + "0.7": 0.051201, + "0.8": 0.051901, + "0.9": 0.053279, + "0.95": 0.054961, + "0.99": 0.057250, + "0.999": 0.060994, + "0.9999": 0.061369 + }, + "queries_per_second": 19.690909, + "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))", + "query_index": 10, + "rows_per_second": 1969090907.890480, + "total_time": 5.078486 + }, + { + "bytes_per_second": 5040506574.582713, + "memory_usage": 0, + "min_time": 0.146000, + "quantiles": { + "0.1": 0.150680, + "0.2": 0.153432, + "0.3": 0.154729, + "0.4": 0.155504, + "0.5": 0.157304, + "0.6": 0.158844, + "0.7": 0.161233, + "0.8": 0.163422, + "0.9": 0.165739, + "0.95": 0.167594, + "0.99": 0.176111, + "0.999": 0.176829, + "0.9999": 0.176901 + }, + "queries_per_second": 6.248239, + "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))", + "query_index": 11, + "rows_per_second": 630063321.822839, + "total_time": 13.283743 + }, + { + "bytes_per_second": 3099244256.989555, + "memory_usage": 0, + "min_time": 0.117000, + "quantiles": { + "0.1": 0.119749, + "0.2": 0.121213, + "0.3": 0.122322, + "0.4": 0.123211, + "0.5": 0.124256, + "0.6": 0.125377, + "0.7": 0.127171, + "0.8": 0.137271, + "0.9": 0.149331, + "0.95": 0.152607, + "0.99": 0.164544, + "0.999": 0.166112, + "0.9999": 0.166269 + }, + "queries_per_second": 7.748111, + "query": "SELECT count() from dt where not ignore(addDays(x, 1))", + "query_index": 12, + "rows_per_second": 774811064.247389, + "total_time": 12.906372 + }, + { + "bytes_per_second": 2547627988.069497, + "memory_usage": 0, + "min_time": 0.296000, + "quantiles": { + "0.1": 0.301883, + "0.2": 0.307249, + "0.3": 0.309049, + "0.4": 0.310217, + "0.5": 0.312118, + "0.6": 0.314809, + "0.7": 0.316388, + "0.8": 0.320717, + "0.9": 0.326411, + "0.95": 0.330368, + "0.99": 0.337533, + "0.999": 0.340139, + "0.9999": 0.340399 + }, + "queries_per_second": 3.139254, + "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))", + "query_index": 13, + "rows_per_second": 318453498.508687, + "total_time": 21.661194 + }, + { + "bytes_per_second": 7684364282.194496, + "memory_usage": 0, + "min_time": 0.048000, + "quantiles": { + "0.1": 0.050091, + "0.2": 0.050477, + "0.3": 0.050682, + "0.4": 0.051107, + "0.5": 0.051671, + "0.6": 0.052214, + "0.7": 0.052887, + "0.8": 0.053680, + "0.9": 0.054681, + "0.95": 0.055761, + "0.99": 0.056652, + "0.999": 0.056818, + "0.9999": 0.056835 + }, + "queries_per_second": 19.210911, + "query": "SELECT sum(x = x) FROM dt", + "query_index": 14, + "rows_per_second": 1921091070.548624, + "total_time": 5.205375 + }, + { + "bytes_per_second": 8695127619.546698, + "memory_usage": 0, + "min_time": 0.086000, + "quantiles": { + "0.1": 0.089082, + "0.2": 0.090057, + "0.3": 0.090563, + "0.4": 0.091275, + "0.5": 0.091653, + "0.6": 0.092261, + "0.7": 0.093265, + "0.8": 0.093866, + "0.9": 0.095030, + "0.95": 0.095763, + "0.99": 0.098011, + "0.999": 0.102204, + "0.9999": 0.102623 + }, + "queries_per_second": 10.868910, + "query": "SELECT sum(x = x) FROM dt64", + "query_index": 15, + "rows_per_second": 1086890952.443337, + "total_time": 9.200555 + }, + { + "bytes_per_second": 992400274.246293, + "memory_usage": 0, + "min_time": 0.378000, + "quantiles": { + "0.1": 0.388084, + "0.2": 0.392829, + "0.3": 0.397263, + "0.4": 0.399389, + "0.5": 0.402049, + "0.6": 0.404498, + "0.7": 0.405675, + "0.8": 0.414647, + "0.9": 0.425558, + "0.95": 0.429786, + "0.99": 0.433592, + "0.999": 0.435479, + "0.9999": 0.435668 + }, + "queries_per_second": 2.418976, + "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt", + "query_index": 16, + "rows_per_second": 248100068.561573, + "total_time": 16.122527 + }, + { + "bytes_per_second": 1029644671.009825, + "memory_usage": 0, + "min_time": 0.737000, + "quantiles": { + "0.1": 0.754697, + "0.2": 0.759189, + "0.3": 0.762915, + "0.4": 0.767468, + "0.5": 0.768731, + "0.6": 0.771708, + "0.7": 0.778973, + "0.8": 0.788201, + "0.9": 0.790550, + "0.95": 0.809304, + "0.99": 0.825193, + "0.999": 0.828099, + "0.9999": 0.828389 + }, + "queries_per_second": 1.279477, + "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64", + "query_index": 17, + "rows_per_second": 128705583.876228, + "total_time": 20.320804 + } + ], + "server_version": "19.18.1", + "test_name": "date_time_64", + "time": "2019-12-11 16:23:52" +} +] diff --git a/dbms/tests/performance/round_down.xml b/dbms/tests/performance/round_down.xml index 353f169ae8d..f453467ab2d 100644 --- a/dbms/tests/performance/round_down.xml +++ b/dbms/tests/performance/round_down.xml @@ -10,9 +10,9 @@ - SELECT count() FROM numbers(10000000) WHERE NOT ignore(roundDuration(rand() % 65536)) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(roundDown(rand() % 65536, [0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000])) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(roundAge(rand() % 100)) - SELECT count() FROM numbers(10000000) WHERE NOT ignore(roundDown(rand() % 100, [0, 1, 18, 25, 35, 45, 55])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(roundDuration(rand() % 65536)) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(roundDown(rand() % 65536, [0, 1, 10, 30, 60, 120, 180, 240, 300, 600, 1200, 1800, 3600, 7200, 18000, 36000])) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(roundAge(rand() % 100)) + SELECT count() FROM zeros(10000000) WHERE NOT ignore(roundDown(rand() % 100, [0, 1, 18, 25, 35, 45, 55])) SELECT count() FROM numbers(10000000) WHERE NOT ignore(roundDown(rand() % 65536, (SELECT groupArray(number) FROM numbers(65536)))) diff --git a/dbms/tests/performance/set.xml b/dbms/tests/performance/set.xml index 8d50dbbce23..345d9c05573 100644 --- a/dbms/tests/performance/set.xml +++ b/dbms/tests/performance/set.xml @@ -17,8 +17,8 @@ table - numbers(10000000) - numbers_mt(100000000) + zeros(10000000) + zeros_mt(100000000) diff --git a/dbms/tests/performance/set_index.xml b/dbms/tests/performance/set_index.xml index 1d1b2460e85..e5848b08124 100644 --- a/dbms/tests/performance/set_index.xml +++ b/dbms/tests/performance/set_index.xml @@ -14,7 +14,7 @@ - SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM zeros(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' DROP TABLE IF EXISTS test_in diff --git a/dbms/tests/performance/sort.xml b/dbms/tests/performance/sort.xml index da7f2fd5410..652dd7f4670 100644 --- a/dbms/tests/performance/sort.xml +++ b/dbms/tests/performance/sort.xml @@ -22,19 +22,19 @@ CREATE TABLE rand_unlimited_10m_64 (key UInt64) Engine = Memory - INSERT INTO rand_unlimited_10m_8 SELECT rand() AS x FROM numbers(10000000) + INSERT INTO rand_unlimited_10m_8 SELECT rand() AS x FROM zeros(10000000) - INSERT INTO rand_1k_10m_16 SELECT rand()%1000 AS x FROM numbers(10000000) - INSERT INTO rand_100k_10m_16 SELECT rand()%100000 AS x FROM numbers(10000000) - INSERT INTO rand_unlimited_10m_16 SELECT rand() AS x FROM numbers(10000000) + INSERT INTO rand_1k_10m_16 SELECT rand()%1000 AS x FROM zeros(10000000) + INSERT INTO rand_100k_10m_16 SELECT rand()%100000 AS x FROM zeros(10000000) + INSERT INTO rand_unlimited_10m_16 SELECT rand() AS x FROM zeros(10000000) - INSERT INTO rand_1k_10m_32 SELECT rand()%1000 AS x FROM numbers(10000000) - INSERT INTO rand_100k_10m_32 SELECT rand()%100000 AS x FROM numbers(10000000) - INSERT INTO rand_unlimited_10m_32 SELECT rand() AS x FROM numbers(10000000) + INSERT INTO rand_1k_10m_32 SELECT rand()%1000 AS x FROM zeros(10000000) + INSERT INTO rand_100k_10m_32 SELECT rand()%100000 AS x FROM zeros(10000000) + INSERT INTO rand_unlimited_10m_32 SELECT rand() AS x FROM zeros(10000000) - INSERT INTO rand_1k_10m_64 SELECT rand()%1000 AS x FROM numbers(10000000) - INSERT INTO rand_100k_10m_64 SELECT rand()%100000 AS x FROM numbers(10000000) - INSERT INTO rand_unlimited_10m_64 SELECT rand() AS x FROM numbers(10000000) + INSERT INTO rand_1k_10m_64 SELECT rand()%1000 AS x FROM zeros(10000000) + INSERT INTO rand_100k_10m_64 SELECT rand()%100000 AS x FROM zeros(10000000) + INSERT INTO rand_unlimited_10m_64 SELECT rand() AS x FROM zeros(10000000) SELECT sum(key) FROM (SELECT * FROM rand_unlimited_10m_8 ORDER BY key) diff --git a/dbms/tests/performance/string_join.xml b/dbms/tests/performance/string_join.xml index 3988845641c..6aa2c576b4e 100644 --- a/dbms/tests/performance/string_join.xml +++ b/dbms/tests/performance/string_join.xml @@ -15,7 +15,7 @@ CREATE TABLE strings (short String, long String) ENGINE Memory INSERT INTO hits_10m_words SELECT DISTINCT arrayJoin(splitByString(' ', SearchPhrase)) AS word, UserID FROM hits_10m_single WHERE length(word) > 0 - INSERT INTO strings SELECT toString(rand()) a, a || a || a || a || a || a || a || a || a || a || a || a FROM numbers(1000000) + INSERT INTO strings SELECT toString(rand()) a, a || a || a || a || a || a || a || a || a || a || a || a FROM zeros(1000000) 1 diff --git a/dbms/tests/performance/string_set.xml b/dbms/tests/performance/string_set.xml index 95612fb2d34..7890ab11a4a 100644 --- a/dbms/tests/performance/string_set.xml +++ b/dbms/tests/performance/string_set.xml @@ -15,7 +15,7 @@ CREATE TABLE strings (short String, long String) ENGINE Memory INSERT INTO hits_10m_words SELECT DISTINCT arrayJoin(splitByString(' ', SearchPhrase)) AS word, UserID FROM hits_10m_single WHERE length(word) > 0 - INSERT INTO strings SELECT toString(rand()) a, a || a || a || a || a || a || a || a || a || a || a || a FROM numbers(1000000) + INSERT INTO strings SELECT toString(rand()) a, a || a || a || a || a || a || a || a || a || a || a || a FROM zeros(1000000) 1 diff --git a/dbms/tests/performance/synthetic_hardware_benchmark.xml b/dbms/tests/performance/synthetic_hardware_benchmark.xml index 055f8f67ee5..9a2048db8da 100644 --- a/dbms/tests/performance/synthetic_hardware_benchmark.xml +++ b/dbms/tests/performance/synthetic_hardware_benchmark.xml @@ -15,8 +15,8 @@ Мы запускаем этот запрос и наблюдаем, с какой скоростью он выполняется. Через несколько секунд, когда скорость стабилизируется, прерываем выполнение. В качестве скорости выполнения запроса указывается количество обработанных исходных (прочитанных из таблицы) данных в единицу времени. Например, в таблице numbers читаемые нами данные - это числа типа UInt64 (8 байт). Если мы обрабатываем миллиард таких чисел в секунду, то отобразится скорость - 8 GB/sec. --> -SELECT count() FROM numbers(100000000) WHERE NOT ignore(rand()) -SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(rand()) +SELECT count() FROM zeros(100000000) WHERE NOT ignore(rand()) +SELECT count() FROM zeros_mt(1600000000) WHERE NOT ignore(rand()) SELECT count() FROM numbers(100000000) WHERE NOT ignore(intHash64(number)) SELECT count() FROM numbers_mt(1600000000) WHERE NOT ignore(intHash64(number)) @@ -56,6 +56,6 @@ -SELECT count() FROM numbers(100000000) WHERE NOT ignore(materialize('xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx') AS s, concat(s,s,s,s,s,s,s,s,s,s) AS t, concat(t,t,t,t,t,t,t,t,t,t) AS u) SETTINGS max_block_size = 1000 +SELECT count() FROM zeros(100000000) WHERE NOT ignore(materialize('xxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx') AS s, concat(s,s,s,s,s,s,s,s,s,s) AS t, concat(t,t,t,t,t,t,t,t,t,t) AS u) SETTINGS max_block_size = 1000 diff --git a/dbms/tests/performance/vectorize_aggregation_combinators.xml b/dbms/tests/performance/vectorize_aggregation_combinators.xml index 49af4ae0f07..c8de9c306a7 100644 --- a/dbms/tests/performance/vectorize_aggregation_combinators.xml +++ b/dbms/tests/performance/vectorize_aggregation_combinators.xml @@ -19,7 +19,7 @@ INSERT INTO array_data SELECT number % 1024, arrayWithConstant(16, number) from numbers(10000000) - SELECT countMerge(v) FROM (SELECT countState() v FROM numbers(1000000000)) FORMAT Null + SELECT countMerge(v) FROM (SELECT countState() v FROM zeros(1000000000)) FORMAT Null SELECT countMerge(v) FROM (SELECT number % 1024 k, countState() v FROM numbers(1000000000) GROUP BY k) FORMAT Null SELECT sumArray(v) FROM array_data FORMAT Null diff --git a/dbms/tests/performance/visit_param_extract_raw.xml b/dbms/tests/performance/visit_param_extract_raw.xml index 5db6b11a5e0..ca46c79c9b5 100644 --- a/dbms/tests/performance/visit_param_extract_raw.xml +++ b/dbms/tests/performance/visit_param_extract_raw.xml @@ -16,5 +16,5 @@ - SELECT count() FROM numbers(2000000) WHERE NOT ignore(visitParamExtractRaw(materialize({param}), 'myparam')) + SELECT count() FROM zeros(2000000) WHERE NOT ignore(visitParamExtractRaw(materialize({param}), 'myparam')) From cad043b11f64d6f58f88431ea66395136b66c28d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Mar 2020 14:25:53 +0300 Subject: [PATCH 13/22] Remove wrong files. --- dbms/tests/performance/out.txt | 141 ---------- dbms/tests/performance/out2.txt | 141 ---------- dbms/tests/performance/res.txt | 465 -------------------------------- dbms/tests/performance/res2.txt | 465 -------------------------------- 4 files changed, 1212 deletions(-) delete mode 100644 dbms/tests/performance/out.txt delete mode 100644 dbms/tests/performance/out2.txt delete mode 100644 dbms/tests/performance/res.txt delete mode 100644 dbms/tests/performance/res2.txt diff --git a/dbms/tests/performance/out.txt b/dbms/tests/performance/out.txt deleted file mode 100644 index 4c6490ef837..00000000000 --- a/dbms/tests/performance/out.txt +++ /dev/null @@ -1,141 +0,0 @@ -var queries = -[ - { - "comment": "", - "query": "SELECT count() FROM dt where not ignore(x)" - }, - { - "comment": "", - "query": "SELECT count() FROM dt64 where not ignore(x)" - }, - { - "comment": "", - "query": "SELECT max(x) FROM dt" - }, - { - "comment": "", - "query": "SELECT max(x) FROM dt64" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toString(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toString(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toStartOfDay(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(addDays(x, 1))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))" - }, - { - "comment": "", - "query": "SELECT sum(x = x) FROM dt" - }, - { - "comment": "", - "query": "SELECT sum(x = x) FROM dt64" - }, - { - "comment": "", - "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt" - }, - { - "comment": "", - "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64" - } -] ; -var results = -[ - { - "system": "ClickHouse", - "comments": "", - "version": "19.18.1", - "result": [ - [ - 0.039 - ], - [ - 0.073 - ], - [ - 0.043 - ], - [ - 0.215 - ], - [ - 0.272 - ], - [ - 0.504 - ], - [ - 0.056 - ], - [ - 0.16 - ], - [ - 0.058 - ], - [ - 0.164 - ], - [ - 0.046 - ], - [ - 0.146 - ], - [ - 0.117 - ], - [ - 0.295 - ], - [ - 0.047 - ], - [ - 0.085 - ], - [ - 0.376 - ], - [ - 0.734 - ] - ], - "time": "2019-12-11 15:23:16", - "data_size": 100000000 - } -] ; diff --git a/dbms/tests/performance/out2.txt b/dbms/tests/performance/out2.txt deleted file mode 100644 index b5f740ea502..00000000000 --- a/dbms/tests/performance/out2.txt +++ /dev/null @@ -1,141 +0,0 @@ -var queries = -[ - { - "comment": "", - "query": "SELECT count() FROM dt where not ignore(x)" - }, - { - "comment": "", - "query": "SELECT count() FROM dt64 where not ignore(x)" - }, - { - "comment": "", - "query": "SELECT max(x) FROM dt" - }, - { - "comment": "", - "query": "SELECT max(x) FROM dt64" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toString(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toString(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toStartOfDay(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))" - }, - { - "comment": "", - "query": "SELECT count() from dt where not ignore(addDays(x, 1))" - }, - { - "comment": "", - "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))" - }, - { - "comment": "", - "query": "SELECT sum(x = x) FROM dt" - }, - { - "comment": "", - "query": "SELECT sum(x = x) FROM dt64" - }, - { - "comment": "", - "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt" - }, - { - "comment": "", - "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64" - } -] ; -var results = -[ - { - "system": "ClickHouse", - "comments": "", - "version": "19.18.1", - "result": [ - [ - 0.039 - ], - [ - 0.074 - ], - [ - 0.044 - ], - [ - 0.222 - ], - [ - 0.281 - ], - [ - 0.529 - ], - [ - 0.06 - ], - [ - 0.178 - ], - [ - 0.06 - ], - [ - 0.173 - ], - [ - 0.048 - ], - [ - 0.146 - ], - [ - 0.117 - ], - [ - 0.296 - ], - [ - 0.048 - ], - [ - 0.086 - ], - [ - 0.378 - ], - [ - 0.737 - ] - ], - "time": "2019-12-11 16:23:52", - "data_size": 100000000 - } -] ; diff --git a/dbms/tests/performance/res.txt b/dbms/tests/performance/res.txt deleted file mode 100644 index f3e04b06347..00000000000 --- a/dbms/tests/performance/res.txt +++ /dev/null @@ -1,465 +0,0 @@ -[ -{ - "hostname": "nikkochetovdev.sas.yp-c.yandex.net", - "main_metric": "min_time", - "num_cores": 32, - "num_threads": 32, - "path": "/home/nik-kochetov/dev/c2/enmk/ClickHouse/dbms/tests/performance/date_time_64.xml", - "ram": 47475228672, - "runs": [ - { - "bytes_per_second": 9409861540.352007, - "memory_usage": 0, - "min_time": 0.039000, - "quantiles": { - "0.1": 0.040199, - "0.2": 0.040653, - "0.3": 0.041213, - "0.4": 0.041642, - "0.5": 0.042083, - "0.6": 0.042603, - "0.7": 0.043306, - "0.8": 0.043760, - "0.9": 0.044983, - "0.95": 0.046143, - "0.99": 0.047486, - "0.999": 0.058256, - "0.9999": 0.059333 - }, - "queries_per_second": 23.524654, - "query": "SELECT count() FROM dt where not ignore(x)", - "query_index": 0, - "rows_per_second": 2352465385.088002, - "total_time": 4.250860 - }, - { - "bytes_per_second": 10384233889.130009, - "memory_usage": 0, - "min_time": 0.073000, - "quantiles": { - "0.1": 0.075317, - "0.2": 0.075790, - "0.3": 0.076077, - "0.4": 0.076334, - "0.5": 0.076805, - "0.6": 0.077249, - "0.7": 0.077603, - "0.8": 0.078142, - "0.9": 0.079281, - "0.95": 0.079756, - "0.99": 0.081960, - "0.999": 0.083226, - "0.9999": 0.083353 - }, - "queries_per_second": 12.980292, - "query": "SELECT count() FROM dt64 where not ignore(x)", - "query_index": 1, - "rows_per_second": 1298029236.141251, - "total_time": 7.703987 - }, - { - "bytes_per_second": 8750237976.940920, - "memory_usage": 0, - "min_time": 0.043000, - "quantiles": { - "0.1": 0.044337, - "0.2": 0.044870, - "0.3": 0.045090, - "0.4": 0.045218, - "0.5": 0.045366, - "0.6": 0.045701, - "0.7": 0.046101, - "0.8": 0.046507, - "0.9": 0.047493, - "0.95": 0.048040, - "0.99": 0.049223, - "0.999": 0.050705, - "0.9999": 0.050854 - }, - "queries_per_second": 21.875595, - "query": "SELECT max(x) FROM dt", - "query_index": 2, - "rows_per_second": 2187559494.235230, - "total_time": 4.571304 - }, - { - "bytes_per_second": 3487337552.818883, - "memory_usage": 0, - "min_time": 0.215000, - "quantiles": { - "0.1": 0.221970, - "0.2": 0.223491, - "0.3": 0.225555, - "0.4": 0.226886, - "0.5": 0.229043, - "0.6": 0.229805, - "0.7": 0.230861, - "0.8": 0.233639, - "0.9": 0.236776, - "0.95": 0.241592, - "0.99": 0.253925, - "0.999": 0.254084, - "0.9999": 0.254100 - }, - "queries_per_second": 4.303441, - "query": "SELECT max(x) FROM dt64", - "query_index": 3, - "rows_per_second": 435917194.102360, - "total_time": 16.498425 - }, - { - "bytes_per_second": 1368202556.460040, - "memory_usage": 0, - "min_time": 0.272000, - "quantiles": { - "0.1": 0.279798, - "0.2": 0.283832, - "0.3": 0.284502, - "0.4": 0.286428, - "0.5": 0.288811, - "0.6": 0.291098, - "0.7": 0.296558, - "0.8": 0.298619, - "0.9": 0.304383, - "0.95": 0.306885, - "0.99": 0.312714, - "0.999": 0.317105, - "0.9999": 0.317544 - }, - "queries_per_second": 3.371624, - "query": "SELECT count() from dt where not ignore(toString(x))", - "query_index": 4, - "rows_per_second": 342050639.115010, - "total_time": 14.533056 - }, - { - "bytes_per_second": 1450828196.929232, - "memory_usage": 0, - "min_time": 0.504000, - "quantiles": { - "0.1": 0.513239, - "0.2": 0.528871, - "0.3": 0.532407, - "0.4": 0.538834, - "0.5": 0.542961, - "0.6": 0.545868, - "0.7": 0.547043, - "0.8": 0.554186, - "0.9": 0.568202, - "0.95": 0.583093, - "0.99": 0.648873, - "0.999": 0.663674, - "0.9999": 0.665154 - }, - "queries_per_second": 1.776884, - "query": "SELECT count() from dt64 where not ignore(toString(x))", - "query_index": 5, - "rows_per_second": 181353524.616154, - "total_time": 10.692875 - }, - { - "bytes_per_second": 6487544575.900535, - "memory_usage": 0, - "min_time": 0.056000, - "quantiles": { - "0.1": 0.058540, - "0.2": 0.059734, - "0.3": 0.060278, - "0.4": 0.060673, - "0.5": 0.061352, - "0.6": 0.062073, - "0.7": 0.062844, - "0.8": 0.063546, - "0.9": 0.064830, - "0.95": 0.066672, - "0.99": 0.067562, - "0.999": 0.068277, - "0.9999": 0.068349 - }, - "queries_per_second": 16.218861, - "query": "SELECT count() from dt where not ignore(toStartOfDay(x))", - "query_index": 6, - "rows_per_second": 1621886143.975134, - "total_time": 6.165661 - }, - { - "bytes_per_second": 4433432853.411924, - "memory_usage": 0, - "min_time": 0.160000, - "quantiles": { - "0.1": 0.168755, - "0.2": 0.174096, - "0.3": 0.175198, - "0.4": 0.177784, - "0.5": 0.179172, - "0.6": 0.180600, - "0.7": 0.183102, - "0.8": 0.185960, - "0.9": 0.189487, - "0.95": 0.190507, - "0.99": 0.193747, - "0.999": 0.195806, - "0.9999": 0.196012 - }, - "queries_per_second": 5.495579, - "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))", - "query_index": 7, - "rows_per_second": 554179106.676491, - "total_time": 12.919475 - }, - { - "bytes_per_second": 6450969380.532378, - "memory_usage": 0, - "min_time": 0.058000, - "quantiles": { - "0.1": 0.059490, - "0.2": 0.060198, - "0.3": 0.060511, - "0.4": 0.060950, - "0.5": 0.061638, - "0.6": 0.062165, - "0.7": 0.063180, - "0.8": 0.063964, - "0.9": 0.064843, - "0.95": 0.065833, - "0.99": 0.067410, - "0.999": 0.067703, - "0.9999": 0.067732 - }, - "queries_per_second": 16.127423, - "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))", - "query_index": 8, - "rows_per_second": 1612742345.133095, - "total_time": 6.200618 - }, - { - "bytes_per_second": 4347265724.547048, - "memory_usage": 0, - "min_time": 0.164000, - "quantiles": { - "0.1": 0.169796, - "0.2": 0.171982, - "0.3": 0.173706, - "0.4": 0.175892, - "0.5": 0.177192, - "0.6": 0.178714, - "0.7": 0.182275, - "0.8": 0.187131, - "0.9": 0.210764, - "0.95": 0.217970, - "0.99": 0.222446, - "0.999": 0.222648, - "0.9999": 0.222668 - }, - "queries_per_second": 5.391733, - "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))", - "query_index": 9, - "rows_per_second": 543408215.568381, - "total_time": 12.240961 - }, - { - "bytes_per_second": 8056446421.959373, - "memory_usage": 0, - "min_time": 0.046000, - "quantiles": { - "0.1": 0.047554, - "0.2": 0.048000, - "0.3": 0.048242, - "0.4": 0.048407, - "0.5": 0.048770, - "0.6": 0.049179, - "0.7": 0.049716, - "0.8": 0.050657, - "0.9": 0.053740, - "0.95": 0.054697, - "0.99": 0.057408, - "0.999": 0.059362, - "0.9999": 0.059557 - }, - "queries_per_second": 20.141116, - "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))", - "query_index": 10, - "rows_per_second": 2014111605.489843, - "total_time": 4.964968 - }, - { - "bytes_per_second": 5111454198.308989, - "memory_usage": 0, - "min_time": 0.146000, - "quantiles": { - "0.1": 0.150252, - "0.2": 0.151915, - "0.3": 0.153284, - "0.4": 0.154656, - "0.5": 0.155617, - "0.6": 0.156039, - "0.7": 0.158277, - "0.8": 0.159162, - "0.9": 0.161681, - "0.95": 0.164258, - "0.99": 0.169050, - "0.999": 0.172889, - "0.9999": 0.173273 - }, - "queries_per_second": 6.324464, - "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))", - "query_index": 11, - "rows_per_second": 638931774.788624, - "total_time": 10.751900 - }, - { - "bytes_per_second": 3227703999.626698, - "memory_usage": 0, - "min_time": 0.117000, - "quantiles": { - "0.1": 0.118657, - "0.2": 0.120602, - "0.3": 0.121634, - "0.4": 0.122749, - "0.5": 0.123620, - "0.6": 0.124714, - "0.7": 0.125342, - "0.8": 0.126662, - "0.9": 0.129363, - "0.95": 0.130856, - "0.99": 0.134388, - "0.999": 0.135462, - "0.9999": 0.135569 - }, - "queries_per_second": 7.983644, - "query": "SELECT count() from dt where not ignore(addDays(x, 1))", - "query_index": 12, - "rows_per_second": 806925999.906674, - "total_time": 10.772023 - }, - { - "bytes_per_second": 2564513384.816551, - "memory_usage": 0, - "min_time": 0.295000, - "quantiles": { - "0.1": 0.300345, - "0.2": 0.304122, - "0.3": 0.305735, - "0.4": 0.307827, - "0.5": 0.308609, - "0.6": 0.309365, - "0.7": 0.311653, - "0.8": 0.315120, - "0.9": 0.317984, - "0.95": 0.321220, - "0.99": 0.345205, - "0.999": 0.364580, - "0.9999": 0.366517 - }, - "queries_per_second": 3.162491, - "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))", - "query_index": 13, - "rows_per_second": 320564173.102069, - "total_time": 15.810323 - }, - { - "bytes_per_second": 7763363653.267178, - "memory_usage": 0, - "min_time": 0.047000, - "quantiles": { - "0.1": 0.049248, - "0.2": 0.050015, - "0.3": 0.050272, - "0.4": 0.050715, - "0.5": 0.051279, - "0.6": 0.051927, - "0.7": 0.052337, - "0.8": 0.052892, - "0.9": 0.053614, - "0.95": 0.055995, - "0.99": 0.057009, - "0.999": 0.059043, - "0.9999": 0.059246 - }, - "queries_per_second": 19.408409, - "query": "SELECT sum(x = x) FROM dt", - "query_index": 14, - "rows_per_second": 1940840913.316794, - "total_time": 5.152406 - }, - { - "bytes_per_second": 8536230820.533304, - "memory_usage": 0, - "min_time": 0.085000, - "quantiles": { - "0.1": 0.089260, - "0.2": 0.090209, - "0.3": 0.090926, - "0.4": 0.092020, - "0.5": 0.093539, - "0.6": 0.094379, - "0.7": 0.095554, - "0.8": 0.096637, - "0.9": 0.098793, - "0.95": 0.100480, - "0.99": 0.103172, - "0.999": 0.110254, - "0.9999": 0.110962 - }, - "queries_per_second": 10.670289, - "query": "SELECT sum(x = x) FROM dt64", - "query_index": 15, - "rows_per_second": 1067028852.566663, - "total_time": 9.371818 - }, - { - "bytes_per_second": 981290094.751588, - "memory_usage": 0, - "min_time": 0.376000, - "quantiles": { - "0.1": 0.390792, - "0.2": 0.392070, - "0.3": 0.395036, - "0.4": 0.398827, - "0.5": 0.404095, - "0.6": 0.408434, - "0.7": 0.412138, - "0.8": 0.417432, - "0.9": 0.423049, - "0.95": 0.437361, - "0.99": 0.455200, - "0.999": 0.458357, - "0.9999": 0.458672 - }, - "queries_per_second": 2.442035, - "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt", - "query_index": 16, - "rows_per_second": 245322523.687897, - "total_time": 25.388659 - }, - { - "bytes_per_second": 1028205501.170973, - "memory_usage": 0, - "min_time": 0.734000, - "quantiles": { - "0.1": 0.759821, - "0.2": 0.762149, - "0.3": 0.765029, - "0.4": 0.772696, - "0.5": 0.778148, - "0.6": 0.783112, - "0.7": 0.786080, - "0.8": 0.797778, - "0.9": 0.806538, - "0.95": 0.808824, - "0.99": 0.816510, - "0.999": 0.818605, - "0.9999": 0.818814 - }, - "queries_per_second": 1.236670, - "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64", - "query_index": 17, - "rows_per_second": 128525687.646372, - "total_time": 20.215586 - } - ], - "server_version": "19.18.1", - "test_name": "date_time_64", - "time": "2019-12-11 15:23:16" -} -] diff --git a/dbms/tests/performance/res2.txt b/dbms/tests/performance/res2.txt deleted file mode 100644 index 217693cbdbd..00000000000 --- a/dbms/tests/performance/res2.txt +++ /dev/null @@ -1,465 +0,0 @@ -[ -{ - "hostname": "nikkochetovdev.sas.yp-c.yandex.net", - "main_metric": "min_time", - "num_cores": 32, - "num_threads": 32, - "path": "/home/nik-kochetov/dev/c2/enmk/ClickHouse/dbms/tests/performance/date_time_64.xml", - "ram": 47475228672, - "runs": [ - { - "bytes_per_second": 9512005376.525969, - "memory_usage": 0, - "min_time": 0.039000, - "quantiles": { - "0.1": 0.039925, - "0.2": 0.040346, - "0.3": 0.040838, - "0.4": 0.041114, - "0.5": 0.041425, - "0.6": 0.041956, - "0.7": 0.042440, - "0.8": 0.043235, - "0.9": 0.044221, - "0.95": 0.045529, - "0.99": 0.050977, - "0.999": 0.057833, - "0.9999": 0.058518 - }, - "queries_per_second": 23.780013, - "query": "SELECT count() FROM dt where not ignore(x)", - "query_index": 0, - "rows_per_second": 2378001344.131492, - "total_time": 4.205212 - }, - { - "bytes_per_second": 10059387738.161827, - "memory_usage": 0, - "min_time": 0.074000, - "quantiles": { - "0.1": 0.076402, - "0.2": 0.077200, - "0.3": 0.078096, - "0.4": 0.078585, - "0.5": 0.079304, - "0.6": 0.079890, - "0.7": 0.080571, - "0.8": 0.082080, - "0.9": 0.083230, - "0.95": 0.083941, - "0.99": 0.084464, - "0.999": 0.084568, - "0.9999": 0.084579 - }, - "queries_per_second": 12.574235, - "query": "SELECT count() FROM dt64 where not ignore(x)", - "query_index": 1, - "rows_per_second": 1257423467.270228, - "total_time": 7.952770 - }, - { - "bytes_per_second": 8551729880.329183, - "memory_usage": 0, - "min_time": 0.044000, - "quantiles": { - "0.1": 0.045402, - "0.2": 0.045667, - "0.3": 0.046041, - "0.4": 0.046223, - "0.5": 0.046410, - "0.6": 0.046785, - "0.7": 0.047063, - "0.8": 0.047522, - "0.9": 0.048746, - "0.95": 0.049961, - "0.99": 0.051003, - "0.999": 0.051011, - "0.9999": 0.051011 - }, - "queries_per_second": 21.379325, - "query": "SELECT max(x) FROM dt", - "query_index": 2, - "rows_per_second": 2137932470.082296, - "total_time": 4.677416 - }, - { - "bytes_per_second": 3302958415.675775, - "memory_usage": 0, - "min_time": 0.222000, - "quantiles": { - "0.1": 0.231731, - "0.2": 0.233341, - "0.3": 0.234829, - "0.4": 0.236819, - "0.5": 0.240804, - "0.6": 0.242230, - "0.7": 0.243392, - "0.8": 0.246855, - "0.9": 0.250680, - "0.95": 0.254090, - "0.99": 0.258972, - "0.999": 0.259751, - "0.9999": 0.259828 - }, - "queries_per_second": 4.094975, - "query": "SELECT max(x) FROM dt64", - "query_index": 3, - "rows_per_second": 412869801.959472, - "total_time": 12.942693 - }, - { - "bytes_per_second": 1274642546.723289, - "memory_usage": 0, - "min_time": 0.281000, - "quantiles": { - "0.1": 0.300769, - "0.2": 0.303915, - "0.3": 0.305376, - "0.4": 0.306813, - "0.5": 0.308576, - "0.6": 0.310090, - "0.7": 0.313469, - "0.8": 0.317862, - "0.9": 0.323054, - "0.95": 0.334075, - "0.99": 0.346969, - "0.999": 0.349500, - "0.9999": 0.349753 - }, - "queries_per_second": 3.153900, - "query": "SELECT count() from dt where not ignore(toString(x))", - "query_index": 4, - "rows_per_second": 318660636.680822, - "total_time": 10.780303 - }, - { - "bytes_per_second": 1428385007.182179, - "memory_usage": 0, - "min_time": 0.529000, - "quantiles": { - "0.1": 0.538657, - "0.2": 0.539444, - "0.3": 0.541329, - "0.4": 0.542470, - "0.5": 0.544262, - "0.6": 0.547909, - "0.7": 0.551176, - "0.8": 0.554685, - "0.9": 0.563354, - "0.95": 0.655805, - "0.99": 0.775895, - "0.999": 0.802141, - "0.9999": 0.804766 - }, - "queries_per_second": 1.722420, - "query": "SELECT count() from dt64 where not ignore(toString(x))", - "query_index": 5, - "rows_per_second": 178548125.897772, - "total_time": 15.675623 - }, - { - "bytes_per_second": 6202344999.343684, - "memory_usage": 0, - "min_time": 0.060000, - "quantiles": { - "0.1": 0.061787, - "0.2": 0.062539, - "0.3": 0.063240, - "0.4": 0.063643, - "0.5": 0.064330, - "0.6": 0.064915, - "0.7": 0.065499, - "0.8": 0.066099, - "0.9": 0.067622, - "0.95": 0.068367, - "0.99": 0.069822, - "0.999": 0.070812, - "0.9999": 0.070911 - }, - "queries_per_second": 15.505862, - "query": "SELECT count() from dt where not ignore(toStartOfDay(x))", - "query_index": 6, - "rows_per_second": 1550586249.835921, - "total_time": 6.449174 - }, - { - "bytes_per_second": 4216802240.766242, - "memory_usage": 0, - "min_time": 0.178000, - "quantiles": { - "0.1": 0.183190, - "0.2": 0.184556, - "0.3": 0.186327, - "0.4": 0.187608, - "0.5": 0.189150, - "0.6": 0.190982, - "0.7": 0.192435, - "0.8": 0.194643, - "0.9": 0.199023, - "0.95": 0.199987, - "0.99": 0.202437, - "0.999": 0.204918, - "0.9999": 0.205166 - }, - "queries_per_second": 5.217217, - "query": "SELECT count() from dt64 where not ignore(toStartOfDay(x))", - "query_index": 7, - "rows_per_second": 527100280.095780, - "total_time": 18.592288 - }, - { - "bytes_per_second": 6196564885.117176, - "memory_usage": 0, - "min_time": 0.060000, - "quantiles": { - "0.1": 0.062199, - "0.2": 0.062682, - "0.3": 0.063260, - "0.4": 0.064251, - "0.5": 0.064656, - "0.6": 0.065146, - "0.7": 0.065637, - "0.8": 0.066203, - "0.9": 0.066740, - "0.95": 0.067316, - "0.99": 0.068686, - "0.999": 0.069785, - "0.9999": 0.069895 - }, - "queries_per_second": 15.491412, - "query": "SELECT count() from dt where not ignore(toStartOfWeek(x))", - "query_index": 8, - "rows_per_second": 1549141221.279294, - "total_time": 6.455189 - }, - { - "bytes_per_second": 4254486514.800510, - "memory_usage": 0, - "min_time": 0.173000, - "quantiles": { - "0.1": 0.177071, - "0.2": 0.179454, - "0.3": 0.181233, - "0.4": 0.182128, - "0.5": 0.183917, - "0.6": 0.185398, - "0.7": 0.189102, - "0.8": 0.194767, - "0.9": 0.199779, - "0.95": 0.202001, - "0.99": 0.215625, - "0.999": 0.223829, - "0.9999": 0.224649 - }, - "queries_per_second": 5.262679, - "query": "SELECT count() from dt64 where not ignore(toStartOfWeek(x))", - "query_index": 9, - "rows_per_second": 531810814.350064, - "total_time": 10.640969 - }, - { - "bytes_per_second": 7876363631.561918, - "memory_usage": 0, - "min_time": 0.048000, - "quantiles": { - "0.1": 0.048766, - "0.2": 0.049063, - "0.3": 0.049456, - "0.4": 0.049926, - "0.5": 0.050485, - "0.6": 0.050856, - "0.7": 0.051201, - "0.8": 0.051901, - "0.9": 0.053279, - "0.95": 0.054961, - "0.99": 0.057250, - "0.999": 0.060994, - "0.9999": 0.061369 - }, - "queries_per_second": 19.690909, - "query": "SELECT count() from dt where not ignore(toRelativeMinuteNum(x))", - "query_index": 10, - "rows_per_second": 1969090907.890480, - "total_time": 5.078486 - }, - { - "bytes_per_second": 5040506574.582713, - "memory_usage": 0, - "min_time": 0.146000, - "quantiles": { - "0.1": 0.150680, - "0.2": 0.153432, - "0.3": 0.154729, - "0.4": 0.155504, - "0.5": 0.157304, - "0.6": 0.158844, - "0.7": 0.161233, - "0.8": 0.163422, - "0.9": 0.165739, - "0.95": 0.167594, - "0.99": 0.176111, - "0.999": 0.176829, - "0.9999": 0.176901 - }, - "queries_per_second": 6.248239, - "query": "SELECT count() from dt64 where not ignore(toRelativeMinuteNum(x))", - "query_index": 11, - "rows_per_second": 630063321.822839, - "total_time": 13.283743 - }, - { - "bytes_per_second": 3099244256.989555, - "memory_usage": 0, - "min_time": 0.117000, - "quantiles": { - "0.1": 0.119749, - "0.2": 0.121213, - "0.3": 0.122322, - "0.4": 0.123211, - "0.5": 0.124256, - "0.6": 0.125377, - "0.7": 0.127171, - "0.8": 0.137271, - "0.9": 0.149331, - "0.95": 0.152607, - "0.99": 0.164544, - "0.999": 0.166112, - "0.9999": 0.166269 - }, - "queries_per_second": 7.748111, - "query": "SELECT count() from dt where not ignore(addDays(x, 1))", - "query_index": 12, - "rows_per_second": 774811064.247389, - "total_time": 12.906372 - }, - { - "bytes_per_second": 2547627988.069497, - "memory_usage": 0, - "min_time": 0.296000, - "quantiles": { - "0.1": 0.301883, - "0.2": 0.307249, - "0.3": 0.309049, - "0.4": 0.310217, - "0.5": 0.312118, - "0.6": 0.314809, - "0.7": 0.316388, - "0.8": 0.320717, - "0.9": 0.326411, - "0.95": 0.330368, - "0.99": 0.337533, - "0.999": 0.340139, - "0.9999": 0.340399 - }, - "queries_per_second": 3.139254, - "query": "SELECT count() from dt64 where not ignore(addDays(x, 1))", - "query_index": 13, - "rows_per_second": 318453498.508687, - "total_time": 21.661194 - }, - { - "bytes_per_second": 7684364282.194496, - "memory_usage": 0, - "min_time": 0.048000, - "quantiles": { - "0.1": 0.050091, - "0.2": 0.050477, - "0.3": 0.050682, - "0.4": 0.051107, - "0.5": 0.051671, - "0.6": 0.052214, - "0.7": 0.052887, - "0.8": 0.053680, - "0.9": 0.054681, - "0.95": 0.055761, - "0.99": 0.056652, - "0.999": 0.056818, - "0.9999": 0.056835 - }, - "queries_per_second": 19.210911, - "query": "SELECT sum(x = x) FROM dt", - "query_index": 14, - "rows_per_second": 1921091070.548624, - "total_time": 5.205375 - }, - { - "bytes_per_second": 8695127619.546698, - "memory_usage": 0, - "min_time": 0.086000, - "quantiles": { - "0.1": 0.089082, - "0.2": 0.090057, - "0.3": 0.090563, - "0.4": 0.091275, - "0.5": 0.091653, - "0.6": 0.092261, - "0.7": 0.093265, - "0.8": 0.093866, - "0.9": 0.095030, - "0.95": 0.095763, - "0.99": 0.098011, - "0.999": 0.102204, - "0.9999": 0.102623 - }, - "queries_per_second": 10.868910, - "query": "SELECT sum(x = x) FROM dt64", - "query_index": 15, - "rows_per_second": 1086890952.443337, - "total_time": 9.200555 - }, - { - "bytes_per_second": 992400274.246293, - "memory_usage": 0, - "min_time": 0.378000, - "quantiles": { - "0.1": 0.388084, - "0.2": 0.392829, - "0.3": 0.397263, - "0.4": 0.399389, - "0.5": 0.402049, - "0.6": 0.404498, - "0.7": 0.405675, - "0.8": 0.414647, - "0.9": 0.425558, - "0.95": 0.429786, - "0.99": 0.433592, - "0.999": 0.435479, - "0.9999": 0.435668 - }, - "queries_per_second": 2.418976, - "query": "SELECT sum(toDateTime(toString(x)) != x) FROM dt", - "query_index": 16, - "rows_per_second": 248100068.561573, - "total_time": 16.122527 - }, - { - "bytes_per_second": 1029644671.009825, - "memory_usage": 0, - "min_time": 0.737000, - "quantiles": { - "0.1": 0.754697, - "0.2": 0.759189, - "0.3": 0.762915, - "0.4": 0.767468, - "0.5": 0.768731, - "0.6": 0.771708, - "0.7": 0.778973, - "0.8": 0.788201, - "0.9": 0.790550, - "0.95": 0.809304, - "0.99": 0.825193, - "0.999": 0.828099, - "0.9999": 0.828389 - }, - "queries_per_second": 1.279477, - "query": "SELECT sum(toDateTime64(toString(x), 3) != x) FROM dt64", - "query_index": 17, - "rows_per_second": 128705583.876228, - "total_time": 20.320804 - } - ], - "server_version": "19.18.1", - "test_name": "date_time_64", - "time": "2019-12-11 16:23:52" -} -] From e63bb9a4a7f30c5daf6b2bd27b4a913a390b3bbe Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Mar 2020 16:36:33 +0300 Subject: [PATCH 14/22] Fix simpliest case of cyclic aliases in QueryNormalizer --- dbms/src/Interpreters/QueryNormalizer.cpp | 17 +++++++++---- .../tests/gtest_cycle_aliases.cpp | 24 +++++++++++++++++++ .../01097_cyclic_defaults.reference | 1 + .../0_stateless/01097_cyclic_defaults.sql | 15 ++++++++++++ 4 files changed, 52 insertions(+), 5 deletions(-) create mode 100644 dbms/src/Interpreters/tests/gtest_cycle_aliases.cpp create mode 100644 dbms/tests/queries/0_stateless/01097_cyclic_defaults.reference create mode 100644 dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index 363d5ba64b1..bb72a9649c6 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -91,14 +91,21 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) return; } + /// We are alias for other column (node.name), but we are alias by + /// ourselves to some other column auto & alias_node = it_alias->second; - /// Let's replace it with the corresponding tree node. - if (current_asts.count(alias_node.get())) + String our_alias_or_name = alias_node->getAliasOrColumnName(); + std::optional our_name = IdentifierSemantic::getColumnName(alias_node); + + String node_alias = ast->tryGetAlias(); + + if (current_asts.count(alias_node.get()) /// We have loop of multiple aliases + || (node.name == our_alias_or_name && our_name && node_alias == *our_name)) /// Our alias points to node.name, direct loop throw Exception("Cyclic aliases", ErrorCodes::CYCLIC_ALIASES); - String my_alias = ast->tryGetAlias(); - if (!my_alias.empty() && my_alias != alias_node->getAliasOrColumnName()) + /// Let's replace it with the corresponding tree node. + if (!node_alias.empty() && node_alias != our_alias_or_name) { /// Avoid infinite recursion here auto opt_name = IdentifierSemantic::getColumnName(alias_node); @@ -108,7 +115,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) { /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. ast = alias_node->clone(); - ast->setAlias(my_alias); + ast->setAlias(node_alias); } } else diff --git a/dbms/src/Interpreters/tests/gtest_cycle_aliases.cpp b/dbms/src/Interpreters/tests/gtest_cycle_aliases.cpp new file mode 100644 index 00000000000..c8037b23d84 --- /dev/null +++ b/dbms/src/Interpreters/tests/gtest_cycle_aliases.cpp @@ -0,0 +1,24 @@ +#include + +#include +#include +#include +#include +#include + +using namespace DB; + +TEST(QueryNormalizer, SimpleCycleAlias) +{ + String query = "a as b, b as a"; + ParserExpressionList parser(false); + ASTPtr ast = parseQuery(parser, query, 0); + + Aliases aliases; + aliases["a"] = parseQuery(parser, "b as a", 0)->children[0]; + aliases["b"] = parseQuery(parser, "a as b", 0)->children[0]; + + Settings settings; + QueryNormalizer::Data normalizer_data(aliases, settings); + EXPECT_THROW(QueryNormalizer(normalizer_data).visit(ast), Exception); +} diff --git a/dbms/tests/queries/0_stateless/01097_cyclic_defaults.reference b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql new file mode 100644 index 00000000000..b0485f6d044 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS table_with_cyclic_defaults; + +CREATE TABLE table_with_cyclic_defaults (a DEFAULT b, b DEFAULT a) ENGINE = Memory; --{serverError 174} + +CREATE TABLE table_with_cyclic_defaults (a DEFAULT b + 1, b DEFAULT a * a) ENGINE = Memory; --{serverError 174} + +CREATE TABLE table_with_cyclic_defaults (a DEFAULT b, b DEFAULT toString(c), c DEFAULT concat(a, '1')) ENGINE = Memory; --{serverError 174} + +CREATE TABLE table_with_cyclic_defaults (a DEFAULT b, b DEFAULT c, c DEFAULT a * b) ENGINE = Memory; --{serverError 174} + +CREATE TABLE table_with_cyclic_defaults (a String DEFAULT b, b String DEFAULT a) ENGINE = Memory; --{serverError 174} + +SELECT 1; + +DROP TABLE IF EXISTS table_with_cyclic_defaults; From 6d7eb583a5a09df09d838ec489210138a9288312 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Mar 2020 16:39:34 +0300 Subject: [PATCH 15/22] Several cases with alters --- dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql index b0485f6d044..1d63038911f 100644 --- a/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql +++ b/dbms/tests/queries/0_stateless/01097_cyclic_defaults.sql @@ -10,6 +10,12 @@ CREATE TABLE table_with_cyclic_defaults (a DEFAULT b, b DEFAULT c, c DEFAULT a * CREATE TABLE table_with_cyclic_defaults (a String DEFAULT b, b String DEFAULT a) ENGINE = Memory; --{serverError 174} +CREATE TABLE table_with_cyclic_defaults (a String) ENGINE = Memory; + +ALTER TABLE table_with_cyclic_defaults ADD COLUMN c String DEFAULT b, ADD COLUMN b String DEFAULT c; --{serverError 174} + +ALTER TABLE table_with_cyclic_defaults ADD COLUMN b String DEFAULT a, MODIFY COLUMN a DEFAULT b; --{serverError 174} + SELECT 1; DROP TABLE IF EXISTS table_with_cyclic_defaults; From 5b56cb196c6414c85f40539803104d5ed5b6b9a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 11 Mar 2020 16:46:51 +0300 Subject: [PATCH 16/22] Fix debug tests. --- dbms/src/Storages/System/StorageSystemZeros.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemZeros.cpp b/dbms/src/Storages/System/StorageSystemZeros.cpp index ae9bf3d0902..aad9baca47e 100644 --- a/dbms/src/Storages/System/StorageSystemZeros.cpp +++ b/dbms/src/Storages/System/StorageSystemZeros.cpp @@ -118,7 +118,7 @@ Pipes StorageSystemZeros::read( for (size_t i = 0; i < num_streams; ++i) { - auto source = std::make_shared(max_block_size, *limit, state); + auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); if (limit && i == 0) source->addTotalRowsApprox(*limit); From 3be5fa13388801acc5f2d774ff962e5744e95375 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 11 Mar 2020 17:29:34 +0300 Subject: [PATCH 17/22] performance comparison --- docker/packager/binary/build.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 77ad69e406b..94615a5a39d 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -28,6 +28,8 @@ then cp -r ../docker/test/performance-comparison/config /output ||: rm /output/unit_tests_dbms ||: rm /output/clickhouse-odbc-bridge ||: + + cp -r ../docker/test/performance-comparison /output/scripts ||: fi # May be set for split build or for performance test. From 50a88885e1e277c5aa69269d22b9c5dc78383253 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Mar 2020 17:38:58 +0300 Subject: [PATCH 18/22] Fix test for parallel drop --- .../0_stateless/00992_system_parts_race_condition_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh index 2199c99e3be..7dbfda90e41 100755 --- a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh +++ b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh @@ -20,7 +20,7 @@ function thread1() function thread2() { - while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done } function thread3() From 181f64b68c4d06b89687ffef670f70be905b9fb4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 11 Mar 2020 18:00:14 +0300 Subject: [PATCH 19/22] Update 00834_kill_mutation.sh --- dbms/tests/queries/0_stateless/00834_kill_mutation.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh index 5b6fc769d6a..b27ef779416 100755 --- a/dbms/tests/queries/0_stateless/00834_kill_mutation.sh +++ b/dbms/tests/queries/0_stateless/00834_kill_mutation.sh @@ -59,7 +59,7 @@ ${CLICKHOUSE_CLIENT} --query="KILL MUTATION WHERE database = 'test' AND table = wait ${CLICKHOUSE_CLIENT} --query="SELECT * FROM test.kill_mutation" -# must be empty +# must always be empty ${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.mutations WHERE table = 'kill_mutation' AND database = 'test' AND is_done = 0" From a01f82478985c862d399b84f793450c43b52eaa5 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Mar 2020 19:47:17 +0300 Subject: [PATCH 20/22] make test more stable --- .../01055_compact_parts_granularity.sh | 31 +++++++++++++++++++ .../01055_compact_parts_granularity.sql | 21 ------------- 2 files changed, 31 insertions(+), 21 deletions(-) create mode 100755 dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh delete mode 100644 dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sql diff --git a/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh b/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh new file mode 100755 index 00000000000..98218dcbb81 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh @@ -0,0 +1,31 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mt_compact" + +# Checks that granularity correctly computed from small parts. + +$CLICKHOUSE_CLIENT -q "CREATE TABLE mt_compact(a Int, s String) ENGINE = MergeTree ORDER BY a + SETTINGS min_rows_for_wide_part = 1000, + index_granularity = 14;" + +$CLICKHOUSE_CLIENT -q "SYSTEM STOP MERGES mt_compact" + +$CLICKHOUSE_CLIENT --max_block_size=1 --min_insert_block_size_rows=1 -q \ + "INSERT INTO mt_compact SELECT number, 'aaa' FROM numbers(100);" + +$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.parts WHERE table = 'mt_compact' AND database = currentDatabase() AND active" +$CLICKHOUSE_CLIENT -q "SYSTEM START MERGES mt_compact" + +# Retry because already started concurrent merges may interrupt optimize +for i in {0..10}; do + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE mt_compact FINAL SETTINGS optimize_throw_if_noop=1" 2>/dev/null + if [ $? -eq 0 ]; then + break + fi +done + +$CLICKHOUSE_CLIENT -q "SELECT count(), sum(marks) FROM system.parts WHERE table = 'mt_compact' AND database = currentDatabase() AND active" +$CLICKHOUSE_CLIENT -q "DROP TABLE mt_compact" diff --git a/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sql b/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sql deleted file mode 100644 index 8828ffc2da8..00000000000 --- a/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sql +++ /dev/null @@ -1,21 +0,0 @@ -drop table if exists mt_compact; - --- Checks that granularity correctly computed from small parts. - -create table mt_compact(a Int, s String) engine = MergeTree order by a -settings min_rows_for_wide_part = 1000, -index_granularity = 14; - -system stop merges mt_compact; -set max_block_size = 1; -set min_insert_block_size_rows=1; -insert into mt_compact select number, 'aaa' from numbers(100); - -select count() from system.parts where table = 'mt_compact' and database = currentDatabase() and active; - -system start merges mt_compact; -optimize table mt_compact final; - -select count(), sum(marks) from system.parts where table = 'mt_compact' and database = currentDatabase() and active; - -drop table mt_compact; From 3729b2c77ae6887647cf05c17332e21857d3a496 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 11 Mar 2020 21:43:46 +0300 Subject: [PATCH 21/22] Update 01055_compact_parts_granularity.sh --- .../tests/queries/0_stateless/01055_compact_parts_granularity.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh b/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh index 98218dcbb81..bb6d3190577 100755 --- a/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh +++ b/dbms/tests/queries/0_stateless/01055_compact_parts_granularity.sh @@ -25,6 +25,7 @@ for i in {0..10}; do if [ $? -eq 0 ]; then break fi + sleep 0.1 done $CLICKHOUSE_CLIENT -q "SELECT count(), sum(marks) FROM system.parts WHERE table = 'mt_compact' AND database = currentDatabase() AND active" From b4cf1410eddd28b849f096b5e9c5f512515e516b Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Wed, 11 Mar 2020 21:13:15 -0300 Subject: [PATCH 22/22] DateTime documentation improvements, added DateTime64 doc (#9562) * Added DateTime64 docs, added info on DateTime timezone conversion, added ClickHouse.Client to list of ADO.NET clients * I accidentally a word * Review notes * Fix space per review notes Co-Authored-By: Ivan Blinkov * More review notes: fixed wrong notes on DateTime, added more extensive info on automatic type conversion * Minor tweaks * Review notes: missing `code` markers * Formatting fix * Silence PR check * fix typo * fix typos * fix typos * removed meaningless sentence. * TZ new description * Update datetime.md * Update datetime.md * Update datetime.md * Update datetime.md * Update datetime64.md * Update datetime64.md * Update datetime64.md * Update datetime64.md Co-authored-by: DarkWanderer Co-authored-by: Ivan Blinkov Co-authored-by: Denis Zhuravlev --- docs/en/data_types/datetime.md | 51 ++++++++--- docs/en/data_types/datetime64.md | 89 ++++++++++++++++++ .../third-party/client_libraries.md | 2 +- .../en/interfaces/third-party/integrations.md | 1 + docs/ru/data_types/datetime.md | 46 ++++++++-- docs/ru/data_types/datetime64.md | 90 +++++++++++++++++++ .../third-party/client_libraries.md | 2 +- .../ru/interfaces/third-party/integrations.md | 1 + 8 files changed, 258 insertions(+), 24 deletions(-) create mode 100644 docs/en/data_types/datetime64.md create mode 100644 docs/ru/data_types/datetime64.md diff --git a/docs/en/data_types/datetime.md b/docs/en/data_types/datetime.md index 1695b0b86e9..fded3caa4e5 100644 --- a/docs/en/data_types/datetime.md +++ b/docs/en/data_types/datetime.md @@ -1,6 +1,6 @@ # DateTime {#data_type-datetime} -Allows to store an instant in time, that can be expressed as a calendar date and a time of a day. `DateTime` allows to take into account time zones for stored values. +Allows to store an instant in time, that can be expressed as a calendar date and a time of a day. Syntax: @@ -12,21 +12,17 @@ Supported range of values: [1970-01-01 00:00:00, 2105-12-31 23:59:59]. Resolution: 1 second. -SELECT toDateTime(0) -FORMAT TSV - -0000-00-00 00:00:00 - - ## Usage Remarks -A moment of time is stored as [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time), independently of time zones and daylight savings. Additionally `DateTime` can store time zone, that affects how `DateTime` values are displayed in text format and how input strings are parsed for storage. The `tzdata` package, containing [IANA Time Zone Database](https://www.iana.org/time-zones), should be installed in the system. Use the `timedatectl list-timezones` command to list timezones known by a local system. +The point in time is saved as a [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time), regardless of the time zone or daylight saving time. Additionally, the `DateTime` type can store time zone that is the same for the entire column, that affects how the values of the `DateTime` type values are displayed in text format and how the values specified as strings are parsed ('2020-01-01 05:00:01'). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. +A list of supported time zones can be found in the [IANA Time Zone Database](https://www.iana.org/time-zones). +The `tzdata` package, containing [IANA Time Zone Database](https://www.iana.org/time-zones), should be installed in the system. Use the `timedatectl list-timezones` command to list timezones known by a local system. You can explicitly set a time zone for `DateTime`-type columns when creating a table. If the time zone isn't set, ClickHouse uses the value of the [timezone](../operations/server_settings/settings.md#server_settings-timezone) parameter in the server settings or the operating system settings at the moment of the ClickHouse server start. The [clickhouse-client](../interfaces/cli.md) applies the server time zone by default if a time zone isn't explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter. -ClickHouse outputs values in `YYYY-MM-DD hh:mm:ss` text format by default. You can change the format with the [formatDateTime](../query_language/functions/date_time_functions.md#formatdatetime) function. +ClickHouse outputs values in `YYYY-MM-DD hh:mm:ss` text format by default. You can change the output with the [formatDateTime](../query_language/functions/date_time_functions.md#formatdatetime) function. When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../operations/settings/settings.md#settings-date_time_input_format) setting. @@ -40,13 +36,13 @@ CREATE TABLE dt `timestamp` DateTime('Europe/Moscow'), `event_id` UInt8 ) -ENGINE = TinyLog +ENGINE = TinyLog; ``` ```sql -INSERT INTO dt Values (1546300800, 1), ('2019-01-01 00:00:00', 2) +INSERT INTO dt Values (1546300800, 1), ('2019-01-01 00:00:00', 2); ``` ```sql -SELECT * FROM dt +SELECT * FROM dt; ``` ```text ┌───────────timestamp─┬─event_id─┐ @@ -55,7 +51,10 @@ SELECT * FROM dt └─────────────────────┴──────────┘ ``` -Unix timestamp `1546300800` represents the `'2019-01-01 00:00:00'` date and time in `Europe/London` (UTC+0) time zone, but the `timestamp` column stores values in the `Europe/Moscow` (UTC+3) timezone, so the value inserted as Unix timestamp is formatted as `2019-01-01 03:00:00`. +* When inserting datetime as an integer, it is treated as Unix Timestamp (UTC). `1546300800` represents `'2019-01-01 00:00:00'` UTC. However, as `timestamp` column has `Europe/Moscow` (UTC+3) timezone specified, when outputting as string the value will be shown as `'2019-01-01 03:00:00'` +* When inserting string value as datetime, it is treated as being in column timezone. `'2019-01-01 00:00:00'` will be treated as being in `Europe/Moscow` timezone and saved as `1546290000`. + +**2.** Filtering on `DateTime` values ```sql SELECT * FROM dt WHERE timestamp = toDateTime('2019-01-01 00:00:00', 'Europe/Moscow') @@ -65,8 +64,17 @@ SELECT * FROM dt WHERE timestamp = toDateTime('2019-01-01 00:00:00', 'Europe/Mos │ 2019-01-01 00:00:00 │ 2 │ └─────────────────────┴──────────┘ ``` +`DateTime` column values can be filtered using a string value in `WHERE` predicate. It will be converted to `DateTime` automatically: +```sql +SELECT * FROM dt WHERE timestamp = '2019-01-01 00:00:00' +``` +```text +┌───────────timestamp─┬─event_id─┐ +│ 2019-01-01 03:00:00 │ 1 │ +└─────────────────────┴──────────┘ +``` -**2.** Getting a time zone for a `DateTime`-type value: +**3.** Getting a time zone for a `DateTime`-type column: ```sql SELECT toDateTime(now(), 'Europe/Moscow') AS column, toTypeName(column) AS x @@ -77,6 +85,21 @@ SELECT toDateTime(now(), 'Europe/Moscow') AS column, toTypeName(column) AS x └─────────────────────┴───────────────────────────┘ ``` +**4.** Timezone conversion + +```sql +SELECT +toDateTime(timestamp, 'Europe/London') as lon_time, +toDateTime(timestamp, 'Europe/Moscow') as mos_time +FROM dt +``` +```text +┌───────────lon_time──┬────────────mos_time─┐ +│ 2019-01-01 00:00:00 │ 2019-01-01 03:00:00 │ +│ 2018-12-31 21:00:00 │ 2019-01-01 00:00:00 │ +└─────────────────────┴─────────────────────┘ +``` + ## See Also - [Type conversion functions](../query_language/functions/type_conversion_functions.md) diff --git a/docs/en/data_types/datetime64.md b/docs/en/data_types/datetime64.md new file mode 100644 index 00000000000..f7feabbb996 --- /dev/null +++ b/docs/en/data_types/datetime64.md @@ -0,0 +1,89 @@ +# DateTime64 {#data_type-datetime64} + +Allows to store an instant in time, that can be expressed as a calendar date and a time of a day, with defined sub-second precision + +Tick size (precision): 10-precision seconds + +Syntax: +```sql +DateTime64(precision, [timezone]) +``` + +Internally, stores data as number of 'ticks' since epoch start (1970-01-01 00:00:00 UTC) as Int64. The tick resolution is determined by the precision parameter. Additionally, the `DateTime64` type can store time zone that is the same for the entire column, that affects how the values of the `DateTime64` type values are displayed in text format and how the values specified as strings are parsed ('2020-01-01 05:00:01.000'). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. See details in [DateTime](datetime.md). + +## Examples + +**1.** Creating a table with `DateTime64`-type column and inserting data into it: + +```sql +CREATE TABLE dt +( + `timestamp` DateTime64(3, 'Europe/Moscow'), + `event_id` UInt8 +) +ENGINE = TinyLog +``` +```sql +INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2) +``` +```sql +SELECT * FROM dt +``` +```text +┌───────────────timestamp─┬─event_id─┐ +│ 2019-01-01 03:00:00.000 │ 1 │ +│ 2019-01-01 00:00:00.000 │ 2 │ +└─────────────────────────┴──────────┘ +``` + +* When inserting datetime as an integer, it is treated as an appropriately scaled Unix Timestamp (UTC). `1546300800000` (with precision 3) represents `'2019-01-01 00:00:00'` UTC. However, as `timestamp` column has `Europe/Moscow` (UTC+3) timezone specified, when outputting as string the value will be shown as `'2019-01-01 03:00:00'` +* When inserting string value as datetime, it is treated as being in column timezone. `'2019-01-01 00:00:00'` will be treated as being in `Europe/Moscow` timezone and stored as `1546290000000`. + +**2.** Filtering on `DateTime64` values + +```sql +SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow') +``` +```text +┌───────────────timestamp─┬─event_id─┐ +│ 2019-01-01 00:00:00.000 │ 2 │ +└─────────────────────────┴──────────┘ +``` +Unlike `DateTime`, `DateTime64` values are not converted from `String` automatically + +**3.** Getting a time zone for a `DateTime64`-type value: + +```sql +SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x +``` +```text +┌──────────────────column─┬─x──────────────────────────────┐ +│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Europe/Moscow') │ +└─────────────────────────┴────────────────────────────────┘ +``` + +**4.** Timezone conversion + +```sql +SELECT +toDateTime64(timestamp, 3, 'Europe/London') as lon_time, +toDateTime64(timestamp, 3, 'Europe/Moscow') as mos_time +FROM dt +``` +```text +┌───────────────lon_time──┬────────────────mos_time─┐ +│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │ +│ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │ +└─────────────────────────┴─────────────────────────┘ +``` + +## See Also + +- [Type conversion functions](../query_language/functions/type_conversion_functions.md) +- [Functions for working with dates and times](../query_language/functions/date_time_functions.md) +- [Functions for working with arrays](../query_language/functions/array_functions.md) +- [The `date_time_input_format` setting](../operations/settings/settings.md#settings-date_time_input_format) +- [The `timezone` server configuration parameter](../operations/server_settings/settings.md#server_settings-timezone) +- [Operators for working with dates and times](../query_language/operators.md#operators-datetime) +- [`Date` data type](date.md) +- [`DateTime` data type](datetime.md) diff --git a/docs/en/interfaces/third-party/client_libraries.md b/docs/en/interfaces/third-party/client_libraries.md index 167c23bd09c..adba376413f 100644 --- a/docs/en/interfaces/third-party/client_libraries.md +++ b/docs/en/interfaces/third-party/client_libraries.md @@ -40,8 +40,8 @@ - [AORM](https://github.com/TanVD/AORM) - C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - Nim diff --git a/docs/en/interfaces/third-party/integrations.md b/docs/en/interfaces/third-party/integrations.md index 80a13ce846d..62ca059fac2 100644 --- a/docs/en/interfaces/third-party/integrations.md +++ b/docs/en/interfaces/third-party/integrations.md @@ -78,6 +78,7 @@ - C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir diff --git a/docs/ru/data_types/datetime.md b/docs/ru/data_types/datetime.md index 90314c1ba7a..ef9498c9871 100644 --- a/docs/ru/data_types/datetime.md +++ b/docs/ru/data_types/datetime.md @@ -1,6 +1,6 @@ # DateTime {#data_type-datetime} -Позволяет хранить момент времени, который может быть представлен как календарная дата и время. `DateTime` позволяет учесть часовой пояс для хранимых значений. +Позволяет хранить момент времени, который может быть представлен как календарная дата и время. Синтаксис: @@ -14,13 +14,15 @@ DateTime([timezone]) ## Использование -Момент времени сохраняется как Unix timestamp, независимо от часового пояса и переходов на летнее/зимнее время. Дополнительно, `DateTime` позволяет хранить часовой пояс, который влияет на то, как буду отображаться значения типа `DateTime` в текстовом виде и как будут парситься входные строки. Список поддержанных временных зон можно найти в [IANA Time Zone Database](https://www.iana.org/time-zones). +Момент времени сохраняется как [Unix timestamp](https://ru.wikipedia.org/wiki/Unix-%D0%B2%D1%80%D0%B5%D0%BC%D1%8F), независимо от часового пояса и переходов на летнее/зимнее время. Дополнительно, тип `DateTime` позволяет хранить часовой пояс, единый для всей колонки, который влияет на то, как будут отображаться значения типа `DateTime` в текстовом виде и как будут парситься значения заданные в виде строк ('2020-01-01 05:00:01'). Часовой пояс не хранится в строках таблицы (выборки), а хранится в метаданных колонки. +Список поддерживаемых временных зон можно найти в [IANA Time Zone Database](https://www.iana.org/time-zones). +Пакет `tzdata`, содержащий [базу данных часовых поясов IANA](https://www.iana.org/time-zones), должен быть установлен в системе. Используйте команду `timedatectl list-timezones` для получения списка часовых поясов, известных локальной системе. Часовой пояс для столбца типа `DateTime` можно в явном виде установить при создании таблицы. Если часовой пояс не установлен, то ClickHouse использует значение параметра [timezone](../operations/server_settings/settings.md#server_settings-timezone), установленное в конфигурации сервера или в настройках операционной системы на момент запуска сервера. Консольный клиент ClickHouse по умолчанию использует часовой пояс сервера, если для значения `DateTime` часовой пояс не был задан в явном виде при инициализации типа данных. Чтобы использовать часовой пояс клиента, запустите [clickhouse-client](../interfaces/cli.md) с параметром `--use_client_time_zone`. -ClickHouse по умолчанию выводит значение в формате `YYYY-MM-DD hh:mm:ss`. Формат можно поменять с помощь функции [formatDateTime](../query_language/functions/date_time_functions.md#formatdatetime). +ClickHouse отображает значения типа `DateTime` в формате `YYYY-MM-DD hh:mm:ss`. Отображение можно поменять с помощью функции [formatDateTime](../query_language/functions/date_time_functions.md#formatdatetime). При вставке данных в ClickHouse, можно использовать различные форматы даты и времени в зависимости от значения настройки [date_time_input_format](../operations/settings/settings.md#settings-date_time_input_format). @@ -28,18 +30,19 @@ ClickHouse по умолчанию выводит значение в форма **1.** Создание таблицы с столбцом типа `DateTime` и вставка данных в неё: +```sql CREATE TABLE dt ( `timestamp` DateTime('Europe/Moscow'), `event_id` UInt8 ) -ENGINE = TinyLog +ENGINE = TinyLog; ``` ```sql -INSERT INTO dt Values (1546300800, 1), ('2019-01-01 00:00:00', 2) +INSERT INTO dt Values (1546300800, 1), ('2019-01-01 00:00:00', 2); ``` ```sql -SELECT * FROM dt +SELECT * FROM dt; ``` ```text ┌───────────timestamp─┬─event_id─┐ @@ -48,7 +51,10 @@ SELECT * FROM dt └─────────────────────┴──────────┘ ``` -Unix timestamp `1546300800` в часовом поясе `Europe/London (UTC+0)` представляет время `'2019-01-01 00:00:00'`, однако столбец `timestamp` хранит время в часовом поясе `Europe/Moscow (UTC+3)`, таким образом значение, вставленное в виде Unix timestamp, представляет время `2019-01-01 03:00:00`. +* При вставке даты-времени как целого числа, оно трактуется как Unix Timestamp (UTC). Unix timestamp `1546300800` в часовом поясе `Europe/London (UTC+0)` представляет время `'2019-01-01 00:00:00'`. Однако, столбец `timestamp` имеет тип `DateTime('Europe/Moscow (UTC+3)')`, так что при выводе в виде строки время отобразится как `2019-01-01 03:00:00`. +* При вставке даты-времени в виде строки, время трактуется соответственно часовому поясу установленному для колонки. `'2019-01-01 00:00:00'` трактуется как время по Москве (и в базу сохраняется `1546290000`) + +**2.** Фильтрация по значениям даты-времени ```sql SELECT * FROM dt WHERE timestamp = toDateTime('2019-01-01 00:00:00', 'Europe/Moscow') @@ -58,8 +64,17 @@ SELECT * FROM dt WHERE timestamp = toDateTime('2019-01-01 00:00:00', 'Europe/Mos │ 2019-01-01 00:00:00 │ 2 │ └─────────────────────┴──────────┘ ``` +Фильтровать по колонке типа `DateTime` можно, указывая строковое значение в фильтре `WHERE`. Конвертация будет выполнена автоматически: +```sql +SELECT * FROM dt WHERE timestamp = '2019-01-01 00:00:00' +``` +```text +┌───────────timestamp─┬─event_id─┐ +│ 2019-01-01 03:00:00 │ 1 │ +└─────────────────────┴──────────┘ +``` -**2.** Получение часового пояса для значения типа `DateTime`: +**3.** Получение часового пояса для колонки типа `DateTime`: ```sql SELECT toDateTime(now(), 'Europe/Moscow') AS column, toTypeName(column) AS x @@ -70,6 +85,21 @@ SELECT toDateTime(now(), 'Europe/Moscow') AS column, toTypeName(column) AS x └─────────────────────┴───────────────────────────┘ ``` +**4.** Конвертация часовых поясов + +```sql +SELECT +toDateTime(timestamp, 'Europe/London') as lon_time, +toDateTime(timestamp, 'Europe/Moscow') as mos_time +FROM dt +``` +```text +┌───────────lon_time──┬────────────mos_time─┐ +│ 2019-01-01 00:00:00 │ 2019-01-01 03:00:00 │ +│ 2018-12-31 21:00:00 │ 2019-01-01 00:00:00 │ +└─────────────────────┴─────────────────────┘ +``` + ## See Also - [Функции преобразования типов](../query_language/functions/type_conversion_functions.md) diff --git a/docs/ru/data_types/datetime64.md b/docs/ru/data_types/datetime64.md new file mode 100644 index 00000000000..96476984f87 --- /dev/null +++ b/docs/ru/data_types/datetime64.md @@ -0,0 +1,90 @@ +# DateTime64 {#data_type-datetime64} + +Позволяет хранить момент времени, который может быть представлен как календарная дата и время, с заданной суб-секундной точностью. + +Размер тика/точность: 10-precision секунд, где precision - целочисленный параметр типа. + +Синтаксис: +```sql +DateTime64(precision, [timezone]) +``` + +Данные хранятся в виде количества 'тиков', прошедших с момента начала эпохи (1970-01-01 00:00:00 UTC), в Int64. Размер тика определяется параметром precision. Дополнительно, тип `DateTime64` позволяет хранить часовой пояс, единый для всей колонки, который влияет на то, как будут отображаться значения типа `DateTime64` в текстовом виде и как будут парситься значения заданные в виде строк ('2020-01-01 05:00:01.000'). Часовой пояс не хранится в строках таблицы (выборки), а хранится в метаданных колонки. Подробнее см. [DateTime](datetime.md). + +## Пример + +**1.** Создание таблицы с столбцом типа `DateTime64` и вставка данных в неё: + +```sql +CREATE TABLE dt +( + `timestamp` DateTime64(3, 'Europe/Moscow'), + `event_id` UInt8 +) +ENGINE = TinyLog +``` +```sql +INSERT INTO dt Values (1546300800000, 1), ('2019-01-01 00:00:00', 2) +``` +```sql +SELECT * FROM dt +``` +```text +┌───────────────timestamp─┬─event_id─┐ +│ 2019-01-01 03:00:00.000 │ 1 │ +│ 2019-01-01 00:00:00.000 │ 2 │ +└─────────────────────────┴──────────┘ +``` + +* При вставке даты-времени как числа (аналогично 'Unix timestamp'), время трактуется как UTC. Unix timestamp `1546300800` в часовом поясе `Europe/London (UTC+0)` представляет время `'2019-01-01 00:00:00'`. Однако, столбец `timestamp` имеет тип `DateTime('Europe/Moscow (UTC+3)')`, так что при выводе в виде строки время отобразится как `2019-01-01 03:00:00`. +* При вставке даты-времени в виде строки, время трактуется соответственно часовому поясу установленному для колонки. `'2019-01-01 00:00:00'` трактуется как время по Москве (и в базу сохраняется `'2018-12-31 21:00:00'` в виде Unix Timestamp) + +**2.** Фильтрация по значениям даты-времени + +```sql +SELECT * FROM dt WHERE timestamp = toDateTime64('2019-01-01 00:00:00', 3, 'Europe/Moscow') +``` +```text +┌───────────────timestamp─┬─event_id─┐ +│ 2019-01-01 00:00:00.000 │ 2 │ +└─────────────────────────┴──────────┘ +``` +В отличие от типа `DateTime`, `DateTime64` не конвертируется из строк автоматически + +**3.** Получение часового пояса для значения типа `DateTime64`: + +```sql +SELECT toDateTime64(now(), 3, 'Europe/Moscow') AS column, toTypeName(column) AS x +``` +```text +┌──────────────────column─┬─x──────────────────────────────┐ +│ 2019-10-16 04:12:04.000 │ DateTime64(3, 'Europe/Moscow') │ +└─────────────────────────┴────────────────────────────────┘ +``` + +**4.** Конвертация часовых поясов + +```sql +SELECT +toDateTime64(timestamp, 3, 'Europe/London') as lon_time, +toDateTime64(timestamp, 3, 'Europe/Moscow') as mos_time +FROM dt +``` +```text +┌───────────────lon_time──┬────────────────mos_time─┐ +│ 2019-01-01 00:00:00.000 │ 2019-01-01 03:00:00.000 │ +│ 2018-12-31 21:00:00.000 │ 2019-01-01 00:00:00.000 │ +└─────────────────────────┴─────────────────────────┘ +``` + +## See Also + +- [Функции преобразования типов](../query_language/functions/type_conversion_functions.md) +- [Функции для работы с датой и временем](../query_language/functions/date_time_functions.md) +- [Функции для работы с массивами](../query_language/functions/array_functions.md) +- [Настройка `date_time_input_format`](../operations/settings/settings.md#settings-date_time_input_format) +- [Конфигурационный параметр сервера `timezone`](../operations/server_settings/settings.md#server_settings-timezone) +- [Операторы для работы с датой и временем](../query_language/operators.md#operators-datetime) +- [Тип данных `Date`](date.md) +- [Тип данных `DateTime`](datetime.md) + diff --git a/docs/ru/interfaces/third-party/client_libraries.md b/docs/ru/interfaces/third-party/client_libraries.md index d911b2a1005..a9a50a8d742 100644 --- a/docs/ru/interfaces/third-party/client_libraries.md +++ b/docs/ru/interfaces/third-party/client_libraries.md @@ -38,8 +38,8 @@ - [AORM](https://github.com/TanVD/AORM) - C# - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) - - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) + - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - Elixir - [clickhousex](https://github.com/appodeal/clickhousex/) - Nim diff --git a/docs/ru/interfaces/third-party/integrations.md b/docs/ru/interfaces/third-party/integrations.md index ab018edf913..fd784d85c03 100644 --- a/docs/ru/interfaces/third-party/integrations.md +++ b/docs/ru/interfaces/third-party/integrations.md @@ -77,6 +77,7 @@ - C# - [ADO.NET](https://docs.microsoft.com/en-us/dotnet/framework/data/adonet/ado-net-overview) - [ClickHouse.Ado](https://github.com/killwort/ClickHouse-Net) + - [ClickHouse.Client](https://github.com/DarkWanderer/ClickHouse.Client) - [ClickHouse.Net](https://github.com/ilyabreev/ClickHouse.Net) - [ClickHouse.Net.Migrations](https://github.com/ilyabreev/ClickHouse.Net.Migrations) - Elixir