From a507f83d8de125a9c1d9487db0df6163648784bd Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 15 Feb 2022 17:12:37 +0300 Subject: [PATCH 01/32] allow unrestricted reads from zookeeper --- .../System/StorageSystemZooKeeper.cpp | 72 ++++++++++++------- 1 file changed, 47 insertions(+), 25 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index f2b2102c7ff..e47df367d1a 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -16,7 +16,7 @@ #include #include #include - +#include namespace DB { @@ -176,18 +176,56 @@ static Paths extractPath(const ASTPtr & query, ContextPtr context) return extractPathImpl(*select.where(), res, context) ? res : Paths(); } +static void addRow(MutableColumns & res_columns, const String& node, const String& path, const Coordination::GetResponse& res) +{ + if (res.error == Coordination::Error::ZNONODE) + return; /// Node was deleted meanwhile. + + const Coordination::Stat & stat = res.stat; + + size_t col_num = 0; + res_columns[col_num++]->insert(node); + res_columns[col_num++]->insert(res.data); + res_columns[col_num++]->insert(stat.czxid); + res_columns[col_num++]->insert(stat.mzxid); + res_columns[col_num++]->insert(UInt64(stat.ctime / 1000)); + res_columns[col_num++]->insert(UInt64(stat.mtime / 1000)); + res_columns[col_num++]->insert(stat.version); + res_columns[col_num++]->insert(stat.cversion); + res_columns[col_num++]->insert(stat.aversion); + res_columns[col_num++]->insert(stat.ephemeralOwner); + res_columns[col_num++]->insert(stat.dataLength); + res_columns[col_num++]->insert(stat.numChildren); + res_columns[col_num++]->insert(stat.pzxid); + res_columns[col_num++]->insert(path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. +} void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { + bool recursive; const Paths & paths = extractPath(query_info.query, context); - if (paths.empty()) - throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN (subquery) in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); + std::deque queue; + + if (paths.empty()) { + recursive = true; + queue.push_back("/"); + } else { + recursive = false; // do not recurse if path was specified + for (const auto & path : paths) + { + queue.push_back(path); + } + } + std::unordered_set paths_corrected; - for (const auto & path : paths) + while (!queue.empty()) { + const String path = queue.front(); + queue.pop_front(); + const String & path_corrected = pathCorrected(path); auto [it, inserted] = paths_corrected.emplace(path_corrected); if (!inserted) /// Do not repeat processing. @@ -207,27 +245,11 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c for (size_t i = 0, size = nodes.size(); i < size; ++i) { auto res = futures[i].get(); - if (res.error == Coordination::Error::ZNONODE) - continue; /// Node was deleted meanwhile. - - const Coordination::Stat & stat = res.stat; - - size_t col_num = 0; - res_columns[col_num++]->insert(nodes[i]); - res_columns[col_num++]->insert(res.data); - res_columns[col_num++]->insert(stat.czxid); - res_columns[col_num++]->insert(stat.mzxid); - res_columns[col_num++]->insert(UInt64(stat.ctime / 1000)); - res_columns[col_num++]->insert(UInt64(stat.mtime / 1000)); - res_columns[col_num++]->insert(stat.version); - res_columns[col_num++]->insert(stat.cversion); - res_columns[col_num++]->insert(stat.aversion); - res_columns[col_num++]->insert(stat.ephemeralOwner); - res_columns[col_num++]->insert(stat.dataLength); - res_columns[col_num++]->insert(stat.numChildren); - res_columns[col_num++]->insert(stat.pzxid); - res_columns[col_num++]->insert( - path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. + addRow(res_columns, nodes[i], path, res); + if (recursive && res.stat.numChildren > 0) + { + queue.push_back(path_part + '/' + nodes[i]); + } } } } From f342c497efadb994d1834cbae6ff52f835388984 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 16 Feb 2022 20:22:03 +0300 Subject: [PATCH 02/32] fix style --- .../System/StorageSystemZooKeeper.cpp | 59 ++++++++++--------- 1 file changed, 30 insertions(+), 29 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index e47df367d1a..4252a3462df 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -18,6 +18,7 @@ #include #include + namespace DB { @@ -176,43 +177,22 @@ static Paths extractPath(const ASTPtr & query, ContextPtr context) return extractPathImpl(*select.where(), res, context) ? res : Paths(); } -static void addRow(MutableColumns & res_columns, const String& node, const String& path, const Coordination::GetResponse& res) -{ - if (res.error == Coordination::Error::ZNONODE) - return; /// Node was deleted meanwhile. - - const Coordination::Stat & stat = res.stat; - - size_t col_num = 0; - res_columns[col_num++]->insert(node); - res_columns[col_num++]->insert(res.data); - res_columns[col_num++]->insert(stat.czxid); - res_columns[col_num++]->insert(stat.mzxid); - res_columns[col_num++]->insert(UInt64(stat.ctime / 1000)); - res_columns[col_num++]->insert(UInt64(stat.mtime / 1000)); - res_columns[col_num++]->insert(stat.version); - res_columns[col_num++]->insert(stat.cversion); - res_columns[col_num++]->insert(stat.aversion); - res_columns[col_num++]->insert(stat.ephemeralOwner); - res_columns[col_num++]->insert(stat.dataLength); - res_columns[col_num++]->insert(stat.numChildren); - res_columns[col_num++]->insert(stat.pzxid); - res_columns[col_num++]->insert(path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. -} void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { - bool recursive; const Paths & paths = extractPath(query_info.query, context); zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); std::deque queue; - if (paths.empty()) { - recursive = true; + bool recursive = true; + if (paths.empty()) + { queue.push_back("/"); - } else { + } + else + { recursive = false; // do not recurse if path was specified for (const auto & path : paths) { @@ -223,7 +203,7 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c std::unordered_set paths_corrected; while (!queue.empty()) { - const String path = queue.front(); + const String path = std::move(queue.front()); queue.pop_front(); const String & path_corrected = pathCorrected(path); @@ -245,7 +225,28 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c for (size_t i = 0, size = nodes.size(); i < size; ++i) { auto res = futures[i].get(); - addRow(res_columns, nodes[i], path, res); + if (res.error == Coordination::Error::ZNONODE) + continue; /// Node was deleted meanwhile. + + const Coordination::Stat & stat = res.stat; + + size_t col_num = 0; + res_columns[col_num++]->insert(nodes[i]); + res_columns[col_num++]->insert(res.data); + res_columns[col_num++]->insert(stat.czxid); + res_columns[col_num++]->insert(stat.mzxid); + res_columns[col_num++]->insert(UInt64(stat.ctime / 1000)); + res_columns[col_num++]->insert(UInt64(stat.mtime / 1000)); + res_columns[col_num++]->insert(stat.version); + res_columns[col_num++]->insert(stat.cversion); + res_columns[col_num++]->insert(stat.aversion); + res_columns[col_num++]->insert(stat.ephemeralOwner); + res_columns[col_num++]->insert(stat.dataLength); + res_columns[col_num++]->insert(stat.numChildren); + res_columns[col_num++]->insert(stat.pzxid); + res_columns[col_num++]->insert( + path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. + if (recursive && res.stat.numChildren > 0) { queue.push_back(path_part + '/' + nodes[i]); From b6bb479c48a534727700f96ca1790de999474999 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Wed, 16 Feb 2022 23:03:44 +0300 Subject: [PATCH 03/32] add setting to enable unrestricted reads from zookeeper --- src/Core/Settings.h | 2 ++ src/Storages/System/StorageSystemZooKeeper.cpp | 5 ++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 25c3ddbe582..12bb5f959d3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -559,6 +559,8 @@ class IColumn; M(Bool, check_table_dependencies, true, "Check that DDL query (such as DROP TABLE or RENAME) will not break dependencies", 0) \ M(Bool, use_local_cache_for_remote_storage, true, "Use local cache for remote storage like HDFS or S3, it's used for remote table engine only", 0) \ \ + M(Bool, allow_unrestricted_reads_from_keeper, false, "Allow unrestricted (w/o condition on path) reads from system.zookeeper table, can be handy, but is not safe for zookeeper", 0) \ + \ /** Experimental functions */ \ M(Bool, allow_experimental_funnel_functions, false, "Enable experimental functions for funnel analysis.", 0) \ M(Bool, allow_experimental_nlp_functions, false, "Enable experimental functions for natural language processing.", 0) \ diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 4252a3462df..8e534a77a60 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -189,7 +189,10 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c bool recursive = true; if (paths.empty()) { - queue.push_back("/"); + if (context->getSettingsRef().allow_unrestricted_reads_from_keeper) + queue.push_back("/"); + else + throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN (subquery) in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); } else { From 0025110a16a7a79899dbabccb9b78727d15c83b8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 19 Feb 2022 13:00:35 +0300 Subject: [PATCH 04/32] Replace exit() with abort() in case of NuRaft errors CI founds [1]: WARNING: ThreadSanitizer: data race (pid=99) Write of size 8 at 0x7b14002192b0 by thread T27: 12 std::__1::map<> 13 DB::OpenedFileCache::~OpenedFileCache() obj-x86_64-linux-gnu/../src/IO/OpenedFileCache.h:27:7 (clickhouse+0xac66de6) 14 cxa_at_exit_wrapper(void*) crtstuff.c (clickhouse+0xaa3646f) 15 decltype(*(std::__1::forward(fp0)).*fp()) std::__1::__invoke() Previous read of size 8 at 0x7b14002192b0 by thread T37 (mutexes: write M732116415018761216): 4 DB::OpenedFileCache::get() obj-x86_64-linux-gnu/../src/IO/OpenedFileCache.h:47:37 (clickhouse+0xac66784) Thread T27 'nuraft_commit' (tid=193, running) created by main thread at: ... Thread T37 'MergeMutate' (tid=204, running) created by main thread at: ... But it also reports that the mutex was already destroyed: Mutex M732116415018761216 is already destroyed. The problem is that [nuraft can call `exit()`](https://github.com/ClickHouse-Extras/NuRaft/blob/1707a7572aa66ec5d0a2dbe2bf5effa3352e6b2d/src/handle_commit.cxx#L157-L158) which will call atexit handlers: 2022.02.17 22:54:03.495450 [ 193 ] {} RaftInstance: background committing thread encounter err Memory limit (total) exceeded: would use 56.56 GiB (attempt to allocate chunk of 8192 bytes), maximum: 55.82 GiB, exiting to protect the system [1]: https://s3.amazonaws.com/clickhouse-test-reports/33057/5a8cf3ac98808dadf125068a33ed9c622998a484/fuzzer_astfuzzertsan,actions//report.html Let's replace exit() with abort() to avoid this. Signed-off-by: Azat Khuzhin --- src/Coordination/KeeperStateManager.cpp | 9 +++++++++ src/Coordination/KeeperStateManager.h | 2 +- 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index c2d4274f972..52d0b0cc881 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -115,6 +115,15 @@ void KeeperStateManager::loadLogStore(uint64_t last_commited_index, uint64_t log log_store->init(last_commited_index, logs_to_keep); } +void KeeperStateManager::system_exit(const int /* exit_code */) +{ + /// NuRaft itself calls exit() which will call atexit handlers + /// and this may lead to an issues in multi-threaded program. + /// + /// Override this with abort(). + abort(); +} + ClusterConfigPtr KeeperStateManager::getLatestConfigFromLogStore() const { auto entry_with_change = log_store->getLatestConfigChange(); diff --git a/src/Coordination/KeeperStateManager.h b/src/Coordination/KeeperStateManager.h index fad76c89503..66037d78a63 100644 --- a/src/Coordination/KeeperStateManager.h +++ b/src/Coordination/KeeperStateManager.h @@ -73,7 +73,7 @@ public: nuraft::ptr get_srv_config() const { return configuration_wrapper.config; } - void system_exit(const int /* exit_code */) override {} + void system_exit(const int exit_code) override; int getPort() const { From 7fe3bef86634e62c3d5540b40f8831fea8a455c9 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 22 Feb 2022 16:51:30 +0300 Subject: [PATCH 05/32] add test for unrestricted zk reads --- src/Common/ZooKeeper/IKeeper.cpp | 2 +- .../System/StorageSystemZooKeeper.cpp | 3 +- ...21_system_zookeeper_unrestricted.reference | 76 +++++++++++++++++++ .../02221_system_zookeeper_unrestricted.sql | 24 ++++++ 4 files changed, 103 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference create mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql diff --git a/src/Common/ZooKeeper/IKeeper.cpp b/src/Common/ZooKeeper/IKeeper.cpp index 4f0c5efe680..70fe33b3f6e 100644 --- a/src/Common/ZooKeeper/IKeeper.cpp +++ b/src/Common/ZooKeeper/IKeeper.cpp @@ -59,7 +59,7 @@ static void addRootPath(String & path, const String & root_path) throw Exception("Path cannot be empty", Error::ZBADARGUMENTS); if (path[0] != '/') - throw Exception("Path must begin with /, got " + path, Error::ZBADARGUMENTS); + throw Exception("Path must begin with /, got path '" + path + "'", Error::ZBADARGUMENTS); if (root_path.empty()) return; diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 8e534a77a60..3f21f563685 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -189,8 +189,9 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c bool recursive = true; if (paths.empty()) { - if (context->getSettingsRef().allow_unrestricted_reads_from_keeper) + if (context->getSettingsRef().allow_unrestricted_reads_from_keeper) { queue.push_back("/"); + } else throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN (subquery) in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); } diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference new file mode 100644 index 00000000000..813f02af4bb --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -0,0 +1,76 @@ +alter_partition_version +alter_partition_version +block_numbers +block_numbers +blocks +blocks +columns +columns +columns +columns +failed_parts +failed_parts +flags +flags +host +host +is_active +is_active +is_lost +is_lost +last_part +last_part +leader_election +leader_election +leader_election-0 +leader_election-0 +log +log +log_pointer +log_pointer +max_processed_insert_time +max_processed_insert_time +metadata +metadata +metadata +metadata +metadata_version +metadata_version +min_unprocessed_insert_time +min_unprocessed_insert_time +mutation_pointer +mutation_pointer +mutations +mutations +nonincrement_block_numbers +nonincrement_block_numbers +parallel +parallel +part_moves_shard +part_moves_shard +parts +parts +pinned_part_uuids +pinned_part_uuids +queue +queue +quorum +quorum +r1 +r1 +replicas +replicas +s1 +s1 +shared +shared +shared +shared +table_shared_id +table_shared_id +temp +temp +zero_copy_hdfs +zero_copy_hdfs +zero_copy_s3 +zero_copy_s3 diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql new file mode 100644 index 00000000000..8bcc36870d0 --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql @@ -0,0 +1,24 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS sample_table; + +SET allow_unrestricted_reads_from_keeper='true'; + +CREATE TABLE sample_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted/{shard}', '{replica}') +ORDER BY tuple(); + +DROP TABLE IF EXISTS sample_table_2; + +CREATE TABLE sample_table_2 ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_2/{shard}', '{replica}') +ORDER BY tuple(); + +SELECT name FROM (SELECT path, name FROM system.zookeeper ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted%'; + +DROP TABLE IF EXISTS sample_table; +DROP TABLE IF EXISTS sample_table_2; From 43475f79bfb04ce99781e352febcadcf35541c39 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 24 Feb 2022 14:06:37 +0800 Subject: [PATCH 06/32] windowview_multi_column_groupby --- src/Storages/WindowView/StorageWindowView.cpp | 99 +++++++++++-------- .../01048_window_view_parser.reference | 36 +++---- ..._window_view_event_tumble_to_asc.reference | 10 +- .../01060_window_view_event_tumble_to_asc.sh | 24 ++--- 4 files changed, 94 insertions(+), 75 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 37c913f58a9..b9eefe9697b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -639,10 +639,44 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( "The first argument of time window function should not be a constant value.", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + ToIdentifierMatcher::Data query_data; + query_data.window_id_name = window_id_name; + query_data.window_id_alias = window_id_alias; + ToIdentifierMatcher::Visitor to_identifier_visitor(query_data); + + ReplaceFunctionNowData time_now_data; + ReplaceFunctionNowVisitor time_now_visitor(time_now_data); + ReplaceFunctionWindowMatcher::Data func_hop_data; + ReplaceFunctionWindowMatcher::Visitor func_window_visitor(func_hop_data); + + DropTableIdentifierMatcher::Data drop_table_identifier_data; + DropTableIdentifierMatcher::Visitor drop_table_identifier_visitor(drop_table_identifier_data); + + + auto visit = [&](const IAST * ast) + { + auto node = ast->clone(); + QueryNormalizer(normalizer_data).visit(node); + /// now() -> ____timestamp + if (is_time_column_func_now) + { + time_now_visitor.visit(node); + function_now_timezone = time_now_data.now_timezone; + } + drop_table_identifier_visitor.visit(node); + /// tumble/hop -> windowID + func_window_visitor.visit(node); + to_identifier_visitor.visit(node); + node->setAlias(""); + return node; + }; + auto new_storage = std::make_shared(); /// storage != nullptr in case create window view with ENGINE syntax if (storage) { + new_storage->set(new_storage->engine, storage->engine->clone()); + if (storage->ttl_table) throw Exception( ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW, @@ -654,46 +688,14 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( "The ENGINE of WindowView must be MergeTree family of table engines " "including the engines with replication support"); - ToIdentifierMatcher::Data query_data; - query_data.window_id_name = window_id_name; - query_data.window_id_alias = window_id_alias; - ToIdentifierMatcher::Visitor to_identifier_visitor(query_data); - - ReplaceFunctionNowData time_now_data; - ReplaceFunctionNowVisitor time_now_visitor(time_now_data); - ReplaceFunctionWindowMatcher::Data func_hop_data; - ReplaceFunctionWindowMatcher::Visitor func_window_visitor(func_hop_data); - - DropTableIdentifierMatcher::Data drop_table_identifier_data; - DropTableIdentifierMatcher::Visitor drop_table_identifier_visitor(drop_table_identifier_data); - - new_storage->set(new_storage->engine, storage->engine->clone()); - - auto visit = [&](const IAST * ast, IAST *& field) - { - if (ast) - { - auto node = ast->clone(); - QueryNormalizer(normalizer_data).visit(node); - /// now() -> ____timestamp - if (is_time_column_func_now) - { - time_now_visitor.visit(node); - function_now_timezone = time_now_data.now_timezone; - } - drop_table_identifier_visitor.visit(node); - /// tumble/hop -> windowID - func_window_visitor.visit(node); - to_identifier_visitor.visit(node); - node->setAlias(""); - new_storage->set(field, node); - } - }; - - visit(storage->partition_by, new_storage->partition_by); - visit(storage->primary_key, new_storage->primary_key); - visit(storage->order_by, new_storage->order_by); - visit(storage->sample_by, new_storage->sample_by); + if (storage->partition_by) + new_storage->set(new_storage->partition_by, visit(storage->partition_by)); + if (storage->primary_key) + new_storage->set(new_storage->primary_key, visit(storage->primary_key)); + if (storage->order_by) + new_storage->set(new_storage->order_by, visit(storage->order_by)); + if (storage->sample_by) + new_storage->set(new_storage->sample_by, visit(storage->sample_by)); if (storage->settings) new_storage->set(new_storage->settings, storage->settings->clone()); @@ -702,8 +704,21 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( { new_storage->set(new_storage->engine, makeASTFunction("AggregatingMergeTree")); - new_storage->set(new_storage->order_by, std::make_shared(window_id_column_name)); - new_storage->set(new_storage->primary_key, std::make_shared(window_id_column_name)); + if (inner_select_query->groupBy()->children.size() == 1) //GROUP BY windowID + { + auto node = visit(inner_select_query->groupBy()->children[0].get()); + new_storage->set(new_storage->order_by, std::make_shared(node->getColumnName())); + } + else + { + auto group_by_function = makeASTFunction("tuple"); + for (auto & child : inner_select_query->groupBy()->children) + { + auto node = visit(child.get()); + group_by_function->arguments->children.push_back(std::make_shared(node->getColumnName())); + } + new_storage->set(new_storage->order_by, group_by_function); + } } auto new_columns = std::make_shared(); diff --git a/tests/queries/0_stateless/01048_window_view_parser.reference b/tests/queries/0_stateless/01048_window_view_parser.reference index c055971bef3..947b68c3a89 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_parser.reference @@ -1,34 +1,34 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1))`\nORDER BY `windowID(timestamp, toIntervalSecond(1))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(1))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (b, `windowID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`plus(a, b)`, `windowID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (b, `windowID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (b, `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`plus(a, b)`, `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nSETTINGS index_granularity = 8192 ||---DATA COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY (b, `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---JOIN--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32),\n `count(mt_2.b)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference index 55c1ee45827..de722f47f08 100644 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference @@ -1,3 +1,7 @@ -3 1990-01-01 12:00:05 -2 1990-01-01 12:00:10 -2 1990-01-01 12:00:15 +1 1 1990-01-01 12:00:05 +1 2 1990-01-01 12:00:05 +1 3 1990-01-01 12:00:05 +1 4 1990-01-01 12:00:10 +1 5 1990-01-01 12:00:10 +1 6 1990-01-01 12:00:15 +1 7 1990-01-01 12:00:15 diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh index e570f405f62..e15ac2b8e1a 100755 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh @@ -10,22 +10,22 @@ DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE TABLE dst(count UInt64, market Int32, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, market Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, market, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid, market; -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, 1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, 2, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, 3, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, 4, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, 5, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, 6, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, 7, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, 8, '1990/01/01 12:00:30'); EOF while true; do - $CLICKHOUSE_CLIENT --query="SELECT count(*) FROM dst" | grep -q "3" && break || sleep .5 ||: + $CLICKHOUSE_CLIENT --query="SELECT count(*) FROM dst" | grep -q "7" && break || sleep .5 ||: done $CLICKHOUSE_CLIENT --query="SELECT * FROM dst ORDER BY w_end;" From 06469eb793d0e8ec65e5d3c4683d01ac583c9e38 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 24 Feb 2022 14:16:24 +0800 Subject: [PATCH 07/32] remove blank row --- src/Storages/WindowView/StorageWindowView.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index b9eefe9697b..a329b01e9f2 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -652,7 +652,6 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( DropTableIdentifierMatcher::Data drop_table_identifier_data; DropTableIdentifierMatcher::Visitor drop_table_identifier_visitor(drop_table_identifier_data); - auto visit = [&](const IAST * ast) { auto node = ast->clone(); From 2d25c79e3744bb736d320dc3439d23560ddc6552 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 24 Feb 2022 13:40:47 +0300 Subject: [PATCH 08/32] analyze select queries from `system.zookeeper` table with `LIKE pattern` and fetch nodes using prefix recursively --- .../System/StorageSystemZooKeeper.cpp | 146 +++++++++++++----- 1 file changed, 104 insertions(+), 42 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 3f21f563685..120a75ed5c3 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include @@ -48,14 +49,23 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() }; } -using Paths = Strings; +/// Type of path to be fetched +enum class ZkPathType +{ + Exact, /// Fetch all nodes under this path + Prefix, /// Fetch all nodes starting with this prefix, recursively (multiple paths may match prefix) + Recurse, /// Fatch all nodes under this path, recursively +}; + +/// List of paths to be feched from zookeeper +using Paths = std::deque>; static String pathCorrected(const String & path) { String path_corrected; /// path should starts with '/', otherwise ZBADARGUMENTS will be thrown in /// ZooKeeper::sendThread and the session will fail. - if (path[0] != '/') + if (path.empty() || path[0] != '/') path_corrected = '/'; path_corrected += path; /// In all cases except the root, path must not end with a slash. @@ -65,7 +75,7 @@ static String pathCorrected(const String & path) } -static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) +static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, bool allow_unrestricted) { const auto * function = elem.as(); if (!function) @@ -74,7 +84,7 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) if (function->name == "and") { for (const auto & child : function->arguments->children) - if (extractPathImpl(*child, res, context)) + if (extractPathImpl(*child, res, context, allow_unrestricted)) return true; return false; @@ -111,7 +121,7 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) set.checkColumnsNumber(1); const auto & set_column = *set.getSetElements()[0]; for (size_t row = 0; row < set_column.size(); ++row) - res.emplace_back(set_column[row].safeGet()); + res.emplace_back(set_column[row].safeGet(), ZkPathType::Exact); } else { @@ -122,12 +132,12 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) if (String str; literal->value.tryGet(str)) { - res.emplace_back(str); + res.emplace_back(str, ZkPathType::Exact); } else if (Tuple tuple; literal->value.tryGet(tuple)) { for (auto element : tuple) - res.emplace_back(element.safeGet()); + res.emplace_back(element.safeGet(), ZkPathType::Exact); } else return false; @@ -157,7 +167,61 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) if (literal->value.getType() != Field::Types::String) return false; - res.emplace_back(literal->value.safeGet()); + res.emplace_back(literal->value.safeGet(), ZkPathType::Exact); + return true; + } + else if (allow_unrestricted && function->name == "like") + { + const ASTIdentifier * ident; + ASTPtr value; + if ((ident = args.children.at(0)->as())) + value = args.children.at(1); + else if ((ident = args.children.at(1)->as())) + value = args.children.at(0); + else + return false; + + if (ident->name() != "path") + return false; + + auto evaluated = evaluateConstantExpressionAsLiteral(value, context); + const auto * literal = evaluated->as(); + if (!literal) + return false; + + if (literal->value.getType() != Field::Types::String) + return false; + + String pattern = literal->value.safeGet(); + bool has_metasymbol = false; + String prefix; // pattern prefix before the first metasymbol occurance + for (size_t i = 0; i < pattern.size(); i++) + { + char c = pattern[i]; + // Handle escaping of metasymbols + if (c == '\\' && i + 1 < pattern.size()) + { + char c2 = pattern[i + 1]; + if (c2 == '_' || c2 == '%') + { + prefix.append(1, c2); + i++; // to skip two bytes + continue; + } + } + + // Stop prefix on the first metasymbols occurance + if (c == '_' || c == '%') + { + has_metasymbol = true; + break; + } + + prefix.append(1, c); + } + + res.emplace_back(prefix, has_metasymbol ? ZkPathType::Prefix : ZkPathType::Exact); + return true; } @@ -167,60 +231,53 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context) /** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause. */ -static Paths extractPath(const ASTPtr & query, ContextPtr context) +static Paths extractPath(const ASTPtr & query, ContextPtr context, bool allow_unrestricted) { const auto & select = query->as(); if (!select.where()) - return Paths(); + return allow_unrestricted ? Paths{{"/", ZkPathType::Recurse}} : Paths(); Paths res; - return extractPathImpl(*select.where(), res, context) ? res : Paths(); + return extractPathImpl(*select.where(), res, context, allow_unrestricted) ? res : Paths(); } void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const { - const Paths & paths = extractPath(query_info.query, context); + Paths paths = extractPath(query_info.query, context, context->getSettingsRef().allow_unrestricted_reads_from_keeper); zkutil::ZooKeeperPtr zookeeper = context->getZooKeeper(); - std::deque queue; - - bool recursive = true; if (paths.empty()) + throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN (subquery) in WHERE clause unless `set allow_unrestricted_reads_from_keeper = 'true'`.", ErrorCodes::BAD_ARGUMENTS); + + std::unordered_set added; + while (!paths.empty()) { - if (context->getSettingsRef().allow_unrestricted_reads_from_keeper) { - queue.push_back("/"); + auto [path, path_type] = std::move(paths.front()); + paths.pop_front(); + + String prefix; + if (path_type == ZkPathType::Prefix) { + prefix = path; + size_t last_slash = prefix.rfind('/'); + path = prefix.substr(0, last_slash == String::npos ? 0 : last_slash); } - else - throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' or path IN ('path1','path2'...) or path IN (subquery) in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); - } - else - { - recursive = false; // do not recurse if path was specified - for (const auto & path : paths) - { - queue.push_back(path); - } - } - - std::unordered_set paths_corrected; - while (!queue.empty()) - { - const String path = std::move(queue.front()); - queue.pop_front(); - - const String & path_corrected = pathCorrected(path); - auto [it, inserted] = paths_corrected.emplace(path_corrected); - if (!inserted) /// Do not repeat processing. - continue; + String path_corrected = pathCorrected(path); zkutil::Strings nodes = zookeeper->getChildren(path_corrected); - String path_part = path_corrected; if (path_part == "/") path_part.clear(); + if (!prefix.empty()) + { + // Remove nodes that do not match specified prefix + nodes.erase(std::remove_if(nodes.begin(), nodes.end(), [&prefix, &path_part] (const String & node) { + return (path_part + '/' + node).substr(0, prefix.size()) != prefix; + }), nodes.end()); + } + std::vector> futures; futures.reserve(nodes.size()); for (const String & node : nodes) @@ -232,6 +289,11 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c if (res.error == Coordination::Error::ZNONODE) continue; /// Node was deleted meanwhile. + // Deduplication + String key = path_part + '/' + nodes[i]; + if (auto [it, inserted] = added.emplace(key); !inserted) + continue; + const Coordination::Stat & stat = res.stat; size_t col_num = 0; @@ -251,9 +313,9 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c res_columns[col_num++]->insert( path); /// This is the original path. In order to process the request, condition in WHERE should be triggered. - if (recursive && res.stat.numChildren > 0) + if (path_type != ZkPathType::Exact && res.stat.numChildren > 0) { - queue.push_back(path_part + '/' + nodes[i]); + paths.emplace_back(key, ZkPathType::Recurse); } } } From 24058c33272f8e86924e38c2aaf024811698d149 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 24 Feb 2022 14:14:46 +0300 Subject: [PATCH 09/32] add test for `system.zookeeper` table unrestricted read with `WHERE path LIKE pattern` clause --- ...stem_zookeeper_unrestricted_like.reference | 153 ++++++++++++++++++ ...221_system_zookeeper_unrestricted_like.sql | 28 ++++ 2 files changed, 181 insertions(+) create mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference create mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference new file mode 100644 index 00000000000..bcfd56258d7 --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -0,0 +1,153 @@ +alter_partition_version +alter_partition_version +block_numbers +block_numbers +blocks +blocks +columns +columns +columns +columns +failed_parts +failed_parts +flags +flags +host +host +is_active +is_active +is_lost +is_lost +last_part +last_part +leader_election +leader_election +leader_election-0 +leader_election-0 +log +log +log_pointer +log_pointer +max_processed_insert_time +max_processed_insert_time +metadata +metadata +metadata +metadata +metadata_version +metadata_version +min_unprocessed_insert_time +min_unprocessed_insert_time +mutation_pointer +mutation_pointer +mutations +mutations +nonincrement_block_numbers +nonincrement_block_numbers +parallel +parallel +part_moves_shard +part_moves_shard +parts +parts +pinned_part_uuids +pinned_part_uuids +queue +queue +quorum +quorum +r1 +r1 +replicas +replicas +s1 +s1 +shared +shared +shared +shared +table_shared_id +table_shared_id +temp +temp +zero_copy_hdfs +zero_copy_hdfs +zero_copy_s3 +zero_copy_s3 +------------------------- +alter_partition_version +alter_partition_version +block_numbers +block_numbers +blocks +blocks +columns +columns +columns +columns +failed_parts +failed_parts +flags +flags +host +host +is_active +is_active +is_lost +is_lost +last_part +last_part +leader_election +leader_election +leader_election-0 +leader_election-0 +log +log +log_pointer +log_pointer +max_processed_insert_time +max_processed_insert_time +metadata +metadata +metadata +metadata +metadata_version +metadata_version +min_unprocessed_insert_time +min_unprocessed_insert_time +mutation_pointer +mutation_pointer +mutations +mutations +nonincrement_block_numbers +nonincrement_block_numbers +parallel +parallel +part_moves_shard +part_moves_shard +parts +parts +pinned_part_uuids +pinned_part_uuids +queue +queue +quorum +quorum +r1 +r1 +replicas +replicas +s1 +s1 +shared +shared +shared +shared +table_shared_id +table_shared_id +temp +temp +zero_copy_hdfs +zero_copy_hdfs +zero_copy_s3 +zero_copy_s3 diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql new file mode 100644 index 00000000000..3c3a59e3be2 --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql @@ -0,0 +1,28 @@ +-- Tags: zookeeper + +DROP TABLE IF EXISTS sample_table; + +SET allow_unrestricted_reads_from_keeper='true'; + +CREATE TABLE sample_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like/{shard}', '{replica}') +ORDER BY tuple(); + +DROP TABLE IF EXISTS sample_table_2; + +CREATE TABLE sample_table_2 ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like_2/{shard}', '{replica}') +ORDER BY tuple(); + +SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse%' ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted_like%'; + +SELECT '-------------------------'; + +SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse/%' ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted_like%'; + +DROP TABLE IF EXISTS sample_table; +DROP TABLE IF EXISTS sample_table_2; From 54ceadd4b3e0d53e3cf0b70dcab8473848e56173 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 27 Feb 2022 18:25:56 +0300 Subject: [PATCH 10/32] Avoid busy polling in keeper while searching for changelog files to delete Fixes: #34534 Signed-off-by: Azat Khuzhin --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index ab86bb7d44a..eb8a724ade9 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -732,7 +732,7 @@ void Changelog::cleanLogThread() while (!log_files_to_delete_queue.isFinishedAndEmpty()) { std::string path; - if (log_files_to_delete_queue.tryPop(path)) + if (log_files_to_delete_queue.pop(path)) { std::error_code ec; if (std::filesystem::remove(path, ec)) From 685de7166dcffa0dd5674f348d062b1581ee37c8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Feb 2022 14:58:56 +0000 Subject: [PATCH 11/32] Fix 02207_s3_content_type --- tests/queries/0_stateless/02207_s3_content_type.reference | 2 +- tests/queries/0_stateless/02207_s3_content_type.sh | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/02207_s3_content_type.reference b/tests/queries/0_stateless/02207_s3_content_type.reference index 2b0a5bcadc2..b015e4a148c 100644 --- a/tests/queries/0_stateless/02207_s3_content_type.reference +++ b/tests/queries/0_stateless/02207_s3_content_type.reference @@ -1,2 +1,2 @@ -ContentLength:2144451 +ContentLength:6888890 ContentType:binary/octet-stream diff --git a/tests/queries/0_stateless/02207_s3_content_type.sh b/tests/queries/0_stateless/02207_s3_content_type.sh index 5ede30e867c..ca75b36c688 100755 --- a/tests/queries/0_stateless/02207_s3_content_type.sh +++ b/tests/queries/0_stateless/02207_s3_content_type.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT --query " -INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/content-type.csv.gz', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10000, s3_truncate_on_insert = 1; +INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/content-type.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10000, s3_truncate_on_insert = 1; " -aws --endpoint-url http://localhost:11111 s3api head-object --bucket test --key content-type.csv.gz | grep Content | sed 's/[ \t,"]*//g' +aws --endpoint-url http://localhost:11111 s3api head-object --bucket test --key content-type.csv | grep Content | sed 's/[ \t,"]*//g' From 399c9c290233e84035ad8343647c33a59f15f81d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Feb 2022 15:09:54 +0000 Subject: [PATCH 12/32] Add order by to 01854_HTTP_dict_decompression --- .../queries/0_stateless/01854_HTTP_dict_decompression.python | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01854_HTTP_dict_decompression.python b/tests/queries/0_stateless/01854_HTTP_dict_decompression.python index 929eaae8067..4f6878665aa 100644 --- a/tests/queries/0_stateless/01854_HTTP_dict_decompression.python +++ b/tests/queries/0_stateless/01854_HTTP_dict_decompression.python @@ -158,7 +158,7 @@ def test_select(dict_name="", schema="word String, counter UInt32", requests=[], COMPRESS_METHOD = requests[i] print(i, COMPRESS_METHOD, ADDING_ENDING, SEND_ENCODING) - check_answers("select * from {}".format(dict_name), answers[i]) + check_answers("SELECT * FROM {} ORDER BY word".format(dict_name), answers[i]) def main(): # first three for encoding, second three for url @@ -171,7 +171,7 @@ def main(): ] # This answers got experemently in non compressed mode and they are correct - answers = ['''This 152\nHello 1\nis 9283\ndata 555\nWorld 2\ntesting 2313213'''] * 5 + answers = ['''Hello 1\nThis 152\nWorld 2\ndata 555\nis 9283\ntesting 2313213'''] * 5 t = start_server(len(insert_requests)) t.start() From 39407dc17b4eb2becbda1955244ca81a9ee99764 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Feb 2022 16:05:57 +0000 Subject: [PATCH 13/32] Skip 01103_check_cpu_instructions_at_startupfor aarch64 --- .../0_stateless/01103_check_cpu_instructions_at_startup.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh index da99a13e97f..9880e2bee57 100755 --- a/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh +++ b/tests/queries/0_stateless/01103_check_cpu_instructions_at_startup.sh @@ -10,6 +10,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # Don't even try to do that. This test should be disabled for sanitizer builds. ${CLICKHOUSE_LOCAL} --query "SELECT max(value LIKE '%sanitize%') FROM system.build_options" | grep -q '1' && echo '@@SKIP@@: Sanitizer build' && exit +${CLICKHOUSE_LOCAL} --query "SELECT value FROM system.build_options WHERE name = 'SYSTEM_PROCESSOR';" | grep -q 'aarch64' && echo '@@SKIP@@: aarch64 build' && exit + command=$(command -v ${CLICKHOUSE_LOCAL}) if ! hash qemu-x86_64-static 2>/dev/null; then From 1560bb0e3d68f28271d6d56791ffb5893d6d6a97 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Feb 2022 16:06:39 +0000 Subject: [PATCH 14/32] Fix 01492_format_readable_quantity --- .../01492_format_readable_quantity.reference | 46 +++++++++---------- .../01492_format_readable_quantity.sql | 2 +- 2 files changed, 24 insertions(+), 24 deletions(-) diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.reference b/tests/queries/0_stateless/01492_format_readable_quantity.reference index e58a1954eee..247063b70a2 100644 --- a/tests/queries/0_stateless/01492_format_readable_quantity.reference +++ b/tests/queries/0_stateless/01492_format_readable_quantity.reference @@ -20,26 +20,26 @@ 178.48 million 178.48 million 178.48 million 485.17 million 485.17 million 485.17 million 1.32 billion 1.32 billion 1.32 billion -3.58 billion 3.58 billion -2.15 billion -9.74 billion 9.74 billion -2.15 billion -26.49 billion 26.49 billion -2.15 billion -72.00 billion 72.00 billion -2.15 billion -195.73 billion 195.73 billion -2.15 billion -532.05 billion 532.05 billion -2.15 billion -1.45 trillion 1.45 trillion -2.15 billion -3.93 trillion 3.93 trillion -2.15 billion -10.69 trillion 10.69 trillion -2.15 billion -29.05 trillion 29.05 trillion -2.15 billion -78.96 trillion 78.96 trillion -2.15 billion -214.64 trillion 214.64 trillion -2.15 billion -583.46 trillion 583.46 trillion -2.15 billion -1.59 quadrillion 1.59 quadrillion -2.15 billion -4.31 quadrillion 4.31 quadrillion -2.15 billion -11.72 quadrillion 11.72 quadrillion -2.15 billion -31.86 quadrillion 31.86 quadrillion -2.15 billion -86.59 quadrillion 86.59 quadrillion -2.15 billion -235.39 quadrillion 235.39 quadrillion -2.15 billion -639.84 quadrillion 639.84 quadrillion -2.15 billion -1739.27 quadrillion 1739.27 quadrillion -2.15 billion -4727.84 quadrillion 4727.84 quadrillion -2.15 billion -12851.60 quadrillion 12851.60 quadrillion -2.15 billion +3.58 billion 3.58 billion 2.15 billion +9.74 billion 9.74 billion 2.15 billion +26.49 billion 26.49 billion 2.15 billion +72.00 billion 72.00 billion 2.15 billion +195.73 billion 195.73 billion 2.15 billion +532.05 billion 532.05 billion 2.15 billion +1.45 trillion 1.45 trillion 2.15 billion +3.93 trillion 3.93 trillion 2.15 billion +10.69 trillion 10.69 trillion 2.15 billion +29.05 trillion 29.05 trillion 2.15 billion +78.96 trillion 78.96 trillion 2.15 billion +214.64 trillion 214.64 trillion 2.15 billion +583.46 trillion 583.46 trillion 2.15 billion +1.59 quadrillion 1.59 quadrillion 2.15 billion +4.31 quadrillion 4.31 quadrillion 2.15 billion +11.72 quadrillion 11.72 quadrillion 2.15 billion +31.86 quadrillion 31.86 quadrillion 2.15 billion +86.59 quadrillion 86.59 quadrillion 2.15 billion +235.39 quadrillion 235.39 quadrillion 2.15 billion +639.84 quadrillion 639.84 quadrillion 2.15 billion +1739.27 quadrillion 1739.27 quadrillion 2.15 billion +4727.84 quadrillion 4727.84 quadrillion 2.15 billion +12851.60 quadrillion 12851.60 quadrillion 2.15 billion diff --git a/tests/queries/0_stateless/01492_format_readable_quantity.sql b/tests/queries/0_stateless/01492_format_readable_quantity.sql index 3931cde49df..93aa570ccc8 100644 --- a/tests/queries/0_stateless/01492_format_readable_quantity.sql +++ b/tests/queries/0_stateless/01492_format_readable_quantity.sql @@ -1,4 +1,4 @@ -WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(x) AS z +WITH round(exp(number), 6) AS x, toUInt64(x) AS y, toInt32(min2(x, 2147483647)) AS z SELECT formatReadableQuantity(x), formatReadableQuantity(y), formatReadableQuantity(z) FROM system.numbers LIMIT 45; From 96390a9263cb5af3d6e42a84988239c9ae87ce32 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 28 Feb 2022 16:38:39 +0000 Subject: [PATCH 15/32] Fix 01016_simhash_minhash --- .../01016_simhash_minhash.reference | 74 +++++++++---------- .../0_stateless/01016_simhash_minhash.sql | 32 ++++---- 2 files changed, 53 insertions(+), 53 deletions(-) diff --git a/tests/queries/0_stateless/01016_simhash_minhash.reference b/tests/queries/0_stateless/01016_simhash_minhash.reference index 3a668e6dcdb..d4fdcfea6a5 100644 --- a/tests/queries/0_stateless/01016_simhash_minhash.reference +++ b/tests/queries/0_stateless/01016_simhash_minhash.reference @@ -50,92 +50,92 @@ (14260447771268573594,3863279269132177973) uniqExact 6 ngramSimHash -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3906262823 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2857686823 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 676648743 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 1012193063 +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2857686823 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3092567843 +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3906262823 ngramSimHashCaseInsensitive +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 2824132391 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2891240999 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3092567591 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 2824132391 ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3908359975 ngramSimHashUTF8 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3159676711 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 2 676648743 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 1012193063 ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 2924795687 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3159676711 ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3897874215 ngramSimHashCaseInsensitiveUTF8 -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3906262823 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3092567591 ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 2824132391 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 2891241255 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 3092567591 +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 3906262823 wordShingleSimHash -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 857724390 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215270 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 991679910 -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 425963587 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215014 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215270 +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 425963587 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 563598566 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 857724390 +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 991679910 wordShingleSimHashCaseInsensitive -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 959182215 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 429118950 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 420713958 ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 421737795 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 429118950 +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 959182215 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 964941252 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 965465540 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 420713958 wordShingleSimHashUTF8 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 857724390 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215270 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 991679910 -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 425963587 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215014 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 404215270 +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 425963587 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 563598566 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 857724390 +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 991679910 wordShingleSimHashCaseInsensitiveUTF8 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 959182215 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 429118950 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 420713958 ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 421737795 +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 429118950 +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 959182215 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 964941252 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 965465540 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 420713958 ngramMinHash -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,17443426065825246292) -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,17443426065825246292) ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (4388091710993602029,17613327300639166679) +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,17443426065825246292) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7962672159337006560,17443426065825246292) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,17443426065825246292) ngramMinHashCaseInsensitive -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,8535005350590298790) -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,8535005350590298790) ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (4388091710993602029,17613327300639166679) +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,8535005350590298790) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7962672159337006560,8535005350590298790) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,8535005350590298790) ngramMinHashUTF8 +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (4388091710993602029,17613327300639166679) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,17443426065825246292) -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,17443426065825246292) -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (4388091710993602029,17613327300639166679) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7962672159337006560,17443426065825246292) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,17443426065825246292) ngramMinHashCaseInsensitiveUTF8 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,8535005350590298790) -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,8535005350590298790) ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (4388091710993602029,17613327300639166679) +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (6021986790841777095,8535005350590298790) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (7962672159337006560,8535005350590298790) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (13225377334870249827,8535005350590298790) wordShingleMinHash -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (18148981179837829400,14581416672396321264) -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (5044918525503962090,12338022931991160906) +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (18148981179837829400,6048943706095721476) +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (18148981179837829400,14581416672396321264) wordShingleMinHashCaseInsensitive -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (15504011608613565061,6048943706095721476) -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (5044918525503962090,3381836163833256482) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (15504011608613565061,6048943706095721476) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (15504011608613565061,14581416672396321264) +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) wordShingleMinHashUTF8 -ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (18148981179837829400,14581416672396321264) -ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (5044918525503962090,12338022931991160906) -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (18148981179837829400,6048943706095721476) -wordShingleMinHashCaseInsensitiveUTF8 -ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (15504011608613565061,6048943706095721476) ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (18148981179837829400,6048943706095721476) +ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (18148981179837829400,14581416672396321264) +wordShingleMinHashCaseInsensitiveUTF8 ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all of your structured data into the system, and it is immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all your structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems.\n:::::::\nClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (using columns after decompression only). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the read / write availability of the system.\nClickHouse is simple and works out of the box. It simplifies all processing of your data: it loads all structured data into the system and immediately becomes available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 3 (5044918525503962090,3381836163833256482) +ClickHouse makes full use of all available hardware to process every request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (only used columns after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid single points of failure. Downtime for one site or the entire data center will not affect the system\'s read and write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they immediately become available for building reports. The SQL dialect allows you to express the desired result without resorting to any non-standard APIs that can be found in some alternative systems. 1 (15504011608613565061,6048943706095721476) ClickHouse makes full use of all available hardware to process each request as quickly as possible. Peak performance for a single query is over 2 terabytes per second (used columns only after unpacking). In a distributed setup, reads are automatically balanced across healthy replicas to avoid increased latency.\nClickHouse supports asynchronous multi-master replication and can be deployed across multiple data centers. All nodes are equal to avoid a single point of failure. Downtime for one site or the entire data center will not affect the system\'s read / write availability.\nClickHouse is simple and works out of the box. It simplifies all the processing of your data: it loads all your structured data into the system, and they are immediately available for building reports. The SQL dialect allows you to express the desired result without resorting to any of the non-standard APIs found in some alternative systems. 1 (15504011608613565061,14581416672396321264) +ClickHouse uses all available hardware to its full potential to process each query as fast as possible. Peak processing performance for a single query stands at more than 2 terabytes per second (after decompression, only used columns). In distributed setup reads are automatically balanced among healthy replicas to avoid increasing latency.\nClickHouse supports multi-master asynchronous replication and can be deployed across multiple datacenters. All nodes are equal, which allows avoiding having single points of failure. Downtime of a single node or the whole datacenter wont affect the systems availability for both reads and writes.\nClickHouse is simple and works out-of-the-box. It streamlines all your data processing: ingest all your structured data into the system and it becomes instantly available for building reports. SQL dialect allows expressing the desired result without involving any custom non-standard API that could be found in some alternative systems. 1 (16224204290372720939,13975393268888698430) diff --git a/tests/queries/0_stateless/01016_simhash_minhash.sql b/tests/queries/0_stateless/01016_simhash_minhash.sql index 01af9451381..1e77b487851 100644 --- a/tests/queries/0_stateless/01016_simhash_minhash.sql +++ b/tests/queries/0_stateless/01016_simhash_minhash.sql @@ -75,38 +75,38 @@ SELECT 'uniqExact', uniqExact(s) FROM defaults; SELECT 'ngramSimHash'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHash(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHash(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramSimHashCaseInsensitive'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashCaseInsensitive(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashCaseInsensitive(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramSimHashUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashUTF8(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashUTF8(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramSimHashCaseInsensitiveUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramSimHashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleSimHash'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHash(s, 2) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHash(s, 2) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleSimHashCaseInsensitive'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashCaseInsensitive(s, 2) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashCaseInsensitive(s, 2) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleSimHashUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashUTF8(s, 2) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashUTF8(s, 2) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleSimHashCaseInsensitiveUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashCaseInsensitiveUTF8(s, 2) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleSimHashCaseInsensitiveUTF8(s, 2) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramMinHash'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHash(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHash(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramMinHashCaseInsensitive'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashCaseInsensitive(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashCaseInsensitive(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramMinHashUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashUTF8(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashUTF8(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'ngramMinHashCaseInsensitiveUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), ngramMinHashCaseInsensitiveUTF8(s) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleMinHash'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHash(s, 2, 3) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHash(s, 2, 3) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleMinHashCaseInsensitive'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashCaseInsensitive(s, 2, 3) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashCaseInsensitive(s, 2, 3) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleMinHashUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashUTF8(s, 2, 3) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashUTF8(s, 2, 3) as h FROM defaults GROUP BY h ORDER BY h; SELECT 'wordShingleMinHashCaseInsensitiveUTF8'; -SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashCaseInsensitiveUTF8(s, 2, 3) as h FROM defaults GROUP BY h; +SELECT arrayStringConcat(groupArray(s), '\n:::::::\n'), count(), wordShingleMinHashCaseInsensitiveUTF8(s, 2, 3) as h FROM defaults GROUP BY h ORDER BY h; SELECT wordShingleSimHash('foobar', 9223372036854775807); -- { serverError 69 } SELECT wordShingleSimHash('foobar', 1001); -- { serverError 69 } From 8bee0531755b95e6ec4a15a7456bf1006f6feb7a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 28 Feb 2022 21:15:06 +0100 Subject: [PATCH 16/32] Remove redundant content in Linux build docs The same information about split builds existed twice in the document. This change merges both places into one. --- docs/en/development/build.md | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index aaa3bdfd043..5379fc37937 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -156,14 +156,6 @@ $ cd ClickHouse $ ./release ``` -## Faster builds for development - -Normally all tools of the ClickHouse bundle, such as `clickhouse-server`, `clickhouse-client` etc., are linked into a single static executable, `clickhouse`. This executable must be re-linked on every change, which might be slow. One common way to improve build time is to use the 'split' build configuration, which builds a separate binary for every tool, and further splits the code into several shared libraries. To enable this tweak, pass the following flags to `cmake`: - -``` --DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 -``` - ## You Don’t Have to Build ClickHouse {#you-dont-have-to-build-clickhouse} ClickHouse is available in pre-built binaries and packages. Binaries are portable and can be run on any Linux flavour. @@ -172,9 +164,9 @@ They are built for stable, prestable and testing releases as long as for every c To find the freshest build from `master`, go to [commits page](https://github.com/ClickHouse/ClickHouse/commits/master), click on the first green checkmark or red cross near commit, and click to the “Details” link right after “ClickHouse Build Check”. -## Split build configuration {#split-build} +## Faster builds for development: Split build configuration {#split-build} -Normally ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary is linked again, which is slow and may be inconvenient for development. There is an alternative configuration which creates dynamically loaded shared libraries instead, allowing faster incremental builds. To use it, add the following flags to your `cmake` invocation: +Normally, ClickHouse is statically linked into a single static `clickhouse` binary with minimal dependencies. This is convenient for distribution, but it means that on every change the entire binary needs to be linked, which is slow and may be inconvenient for development. There is an alternative configuration which instead creates dynamically loaded shared libraries and separate binaries `clickhouse-server`, `clickhouse-client` etc., allowing for faster incremental builds. To use it, add the following flags to your `cmake` invocation: ``` -DUSE_STATIC_LIBRARIES=0 -DSPLIT_SHARED_LIBRARIES=1 -DCLICKHOUSE_SPLIT_BINARY=1 ``` From a7c6d1153222849f1a44600bc562f490b5f4f3e5 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Mar 2022 11:03:26 +0000 Subject: [PATCH 17/32] Fix schema inference for unquoted dates in CSV --- src/Formats/EscapingRuleUtils.cpp | 2 +- ...02228_unquoted_dates_in_csv_schema_inference.reference | 1 + .../02228_unquoted_dates_in_csv_schema_inference.sh | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference create mode 100755 tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.sh diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 0a7747fc864..b0ea10abdb6 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -262,7 +262,7 @@ static bool evaluateConstantExpressionFromString(const StringRef & field, DataTy /// FIXME: Our parser cannot parse maps in the form of '{key : value}' that is used in text formats. bool parsed = parser.parse(token_iterator, ast, expected); - if (!parsed) + if (!parsed || !token_iterator->isEnd()) return false; try diff --git a/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference new file mode 100644 index 00000000000..5fd48ae580a --- /dev/null +++ b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.reference @@ -0,0 +1 @@ +c1 Nullable(String) diff --git a/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.sh b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.sh new file mode 100755 index 00000000000..314a60d6491 --- /dev/null +++ b/tests/queries/0_stateless/02228_unquoted_dates_in_csv_schema_inference.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "2020-02-01 16:00:00" | $CLICKHOUSE_LOCAL -q "desc table table" --input-format "CSV" --file=- + From 932fa19ecdd22a1c4f3d5eda83d6bf46ec6d52ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Mar 2022 13:17:34 +0100 Subject: [PATCH 18/32] Disable memory checking test with thread sanitizer --- .../queries/0_stateless/01281_group_by_limit_memory_tracking.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index bf201187f45..ad3fa91e13f 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-fasttest +# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan # Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently # Regression for MemoryTracker that had been incorrectly accounted From 673ddd98dedf6175e64fcb2f4563a724be1969f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Mar 2022 20:02:43 +0100 Subject: [PATCH 19/32] Fix tests --- ...21_system_zookeeper_unrestricted.reference | 6 +- .../02221_system_zookeeper_unrestricted.sh | 30 +++++++ .../02221_system_zookeeper_unrestricted.sql | 24 ------ ...stem_zookeeper_unrestricted_like.reference | 82 +------------------ ...2221_system_zookeeper_unrestricted_like.sh | 32 ++++++++ ...221_system_zookeeper_unrestricted_like.sql | 28 ------- 6 files changed, 66 insertions(+), 136 deletions(-) create mode 100755 tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh delete mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql create mode 100755 tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh delete mode 100644 tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index 813f02af4bb..bd0c9cee464 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -1,3 +1,5 @@ +1 +1 alter_partition_version alter_partition_version block_numbers @@ -56,12 +58,8 @@ queue queue quorum quorum -r1 -r1 replicas replicas -s1 -s1 shared shared shared diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh new file mode 100755 index 00000000000..92951535157 --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash +# Tags: zookeeper + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" + +${CLICKHOUSE_CLIENT} -n -q" +CREATE TABLE sample_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted', '1') +ORDER BY tuple(); +" + +${CLICKHOUSE_CLIENT} -n -q" +CREATE TABLE sample_table_2 ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_2', '1') +ORDER BY tuple(); +" + +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query "SELECT name FROM (SELECT path, name FROM system.zookeeper ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted%'"; + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql deleted file mode 100644 index 8bcc36870d0..00000000000 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sql +++ /dev/null @@ -1,24 +0,0 @@ --- Tags: zookeeper - -DROP TABLE IF EXISTS sample_table; - -SET allow_unrestricted_reads_from_keeper='true'; - -CREATE TABLE sample_table ( - key UInt64 -) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted/{shard}', '{replica}') -ORDER BY tuple(); - -DROP TABLE IF EXISTS sample_table_2; - -CREATE TABLE sample_table_2 ( - key UInt64 -) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_2/{shard}', '{replica}') -ORDER BY tuple(); - -SELECT name FROM (SELECT path, name FROM system.zookeeper ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted%'; - -DROP TABLE IF EXISTS sample_table; -DROP TABLE IF EXISTS sample_table_2; diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index bcfd56258d7..f95d60dc07b 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -1,153 +1,75 @@ -alter_partition_version +1 alter_partition_version block_numbers -block_numbers -blocks blocks columns columns -columns -columns -failed_parts failed_parts flags -flags -host host is_active -is_active -is_lost is_lost last_part -last_part -leader_election leader_election leader_election-0 -leader_election-0 -log log log_pointer -log_pointer -max_processed_insert_time max_processed_insert_time metadata metadata -metadata -metadata -metadata_version metadata_version min_unprocessed_insert_time -min_unprocessed_insert_time -mutation_pointer mutation_pointer mutations -mutations -nonincrement_block_numbers nonincrement_block_numbers parallel -parallel -part_moves_shard part_moves_shard parts -parts -pinned_part_uuids pinned_part_uuids queue -queue quorum -quorum -r1 -r1 replicas -replicas -s1 -s1 -shared -shared shared shared table_shared_id -table_shared_id -temp temp zero_copy_hdfs -zero_copy_hdfs -zero_copy_s3 zero_copy_s3 ------------------------- -alter_partition_version +1 alter_partition_version block_numbers -block_numbers -blocks blocks columns columns -columns -columns -failed_parts failed_parts flags -flags -host host is_active -is_active -is_lost is_lost last_part -last_part -leader_election leader_election leader_election-0 -leader_election-0 -log log log_pointer -log_pointer -max_processed_insert_time max_processed_insert_time metadata metadata -metadata -metadata -metadata_version metadata_version min_unprocessed_insert_time -min_unprocessed_insert_time -mutation_pointer mutation_pointer mutations -mutations -nonincrement_block_numbers nonincrement_block_numbers parallel -parallel -part_moves_shard part_moves_shard parts -parts -pinned_part_uuids pinned_part_uuids queue -queue quorum -quorum -r1 -r1 replicas -replicas -s1 -s1 -shared -shared shared shared table_shared_id -table_shared_id -temp temp zero_copy_hdfs -zero_copy_hdfs -zero_copy_s3 zero_copy_s3 diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh new file mode 100755 index 00000000000..8b90d4138d3 --- /dev/null +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -0,0 +1,32 @@ +#!/usr/bin/env bash +# Tags: zookeeper + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" + +${CLICKHOUSE_CLIENT} -n --query="CREATE TABLE sample_table ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like', '1') +ORDER BY tuple(); +DROP TABLE IF EXISTS sample_table;" + + +${CLICKHOUSE_CLIENT} -n --query "CREATE TABLE sample_table_2 ( + key UInt64 +) +ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like_2', '1') +ORDER BY tuple();" + +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_like%'" + +${CLICKHOUSE_CLIENT} --query="SELECT '-------------------------'" + +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse/%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_like%'" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql deleted file mode 100644 index 3c3a59e3be2..00000000000 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sql +++ /dev/null @@ -1,28 +0,0 @@ --- Tags: zookeeper - -DROP TABLE IF EXISTS sample_table; - -SET allow_unrestricted_reads_from_keeper='true'; - -CREATE TABLE sample_table ( - key UInt64 -) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like/{shard}', '{replica}') -ORDER BY tuple(); - -DROP TABLE IF EXISTS sample_table_2; - -CREATE TABLE sample_table_2 ( - key UInt64 -) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like_2/{shard}', '{replica}') -ORDER BY tuple(); - -SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse%' ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted_like%'; - -SELECT '-------------------------'; - -SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse/%' ORDER BY name) WHERE path LIKE '%02221_system_zookeeper_unrestricted_like%'; - -DROP TABLE IF EXISTS sample_table; -DROP TABLE IF EXISTS sample_table_2; From c82ca34df963f82a0ad5fed336d976a1ef735dd9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 1 Mar 2022 20:15:43 +0100 Subject: [PATCH 20/32] Move codebrowser out of Yandex --- README.md | 2 +- docs/en/development/browse-code.md | 2 +- docs/en/whats-new/changelog/2020.md | 2 +- docs/ja/development/browse-code.md | 2 +- docs/ru/development/browse-code.md | 2 +- docs/zh/changelog/index.md | 2 +- docs/zh/development/browse-code.md | 2 +- docs/zh/whats-new/changelog/2020.md | 2 +- website/sitemap-static.xml | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/README.md b/README.md index f433b457861..e6dc9f1e6fc 100644 --- a/README.md +++ b/README.md @@ -10,6 +10,6 @@ ClickHouse® is an open-source column-oriented database management system that a * [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format. * [Slack](https://join.slack.com/t/clickhousedb/shared_invite/zt-rxm3rdrk-lIUmhLC3V8WTaL0TGxsOmg) and [Telegram](https://telegram.me/clickhouse_en) allow chatting with ClickHouse users in real-time. * [Blog](https://clickhouse.com/blog/en/) contains various ClickHouse-related articles, as well as announcements and reports about events. -* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/html_report/ClickHouse/index.html) with syntax highlight and navigation. +* [Code Browser (Woboq)](https://clickhouse.com/codebrowser/ClickHouse/index.html) with syntax highlight and navigation. * [Code Browser (github.dev)](https://github.dev/ClickHouse/ClickHouse) with syntax highlight, powered by github.dev. * [Contacts](https://clickhouse.com/company/#contact) can help to get your questions answered if there are any. diff --git a/docs/en/development/browse-code.md b/docs/en/development/browse-code.md index fa57d2289b3..0fe8a46873c 100644 --- a/docs/en/development/browse-code.md +++ b/docs/en/development/browse-code.md @@ -5,7 +5,7 @@ toc_title: Source Code Browser # Browse ClickHouse Source Code {#browse-clickhouse-source-code} -You can use **Woboq** online code browser available [here](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily. +You can use **Woboq** online code browser available [here](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). It provides code navigation and semantic highlighting, search and indexing. The code snapshot is updated daily. Also, you can browse sources on [GitHub](https://github.com/ClickHouse/ClickHouse) as usual. diff --git a/docs/en/whats-new/changelog/2020.md b/docs/en/whats-new/changelog/2020.md index b54fbacd2b0..e0afe256777 100644 --- a/docs/en/whats-new/changelog/2020.md +++ b/docs/en/whats-new/changelog/2020.md @@ -2968,7 +2968,7 @@ No changes compared to v20.4.3.16-stable. * Updated checking for hung queries in clickhouse-test script [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) * Removed some useless files from repository. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Changed type of math perftests from `once` to `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html) +* Add docker image which allows to build interactive code browser HTML report for our codebase. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/ClickHouse/dbms/index.html) * Suppress some test failures under MSan. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) * Speedup "exception while insert" test. This test often time out in debug-with-coverage build. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Updated `libcxx` and `libcxxabi` to master. In preparation to [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) diff --git a/docs/ja/development/browse-code.md b/docs/ja/development/browse-code.md index 6539014eaf0..2f4cfb622c6 100644 --- a/docs/ja/development/browse-code.md +++ b/docs/ja/development/browse-code.md @@ -7,7 +7,7 @@ toc_title: "\u30BD\u30FC\u30B9\u30B3\u30FC\u30C9\u306E\u53C2\u7167" # ClickHouseのソースコードを参照 {#browse-clickhouse-source-code} -以下を使用できます **Woboq** オンラインのコードブラウザをご利用 [ここに](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). このコードナビゲーションや意味のハイライト表示、検索インデックス. コードのスナップショットは随時更新中です。 +以下を使用できます **Woboq** オンラインのコードブラウザをご利用 [ここに](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). このコードナビゲーションや意味のハイライト表示、検索インデックス. コードのスナップショットは随時更新中です。 また、ソースを参照することもできます [GitHub](https://github.com/ClickHouse/ClickHouse) いつものように diff --git a/docs/ru/development/browse-code.md b/docs/ru/development/browse-code.md index 26b3f491599..730e97aed27 100644 --- a/docs/ru/development/browse-code.md +++ b/docs/ru/development/browse-code.md @@ -6,7 +6,7 @@ toc_title: "Навигация по коду ClickHouse" # Навигация по коду ClickHouse {#navigatsiia-po-kodu-clickhouse} -Для навигации по коду онлайн доступен **Woboq**, он расположен [здесь](https://clickhouse.com/codebrowser/html_report///ClickHouse/src/index.html). В нём реализовано удобное перемещение между исходными файлами, семантическая подсветка, подсказки, индексация и поиск. Слепок кода обновляется ежедневно. +Для навигации по коду онлайн доступен **Woboq**, он расположен [здесь](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). В нём реализовано удобное перемещение между исходными файлами, семантическая подсветка, подсказки, индексация и поиск. Слепок кода обновляется ежедневно. Также вы можете просматривать исходники на [GitHub](https://github.com/ClickHouse/ClickHouse). diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index d36a676134e..306c72103fb 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -247,7 +247,7 @@ toc_title: "\u53D8\u66F4\u65E5\u5FD7" - 更新了clickhouse-test脚本中挂起查询的检查 [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([亚历山大\*卡扎科夫](https://github.com/Akazz)) - 从存储库中删除了一些无用的文件。 [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 更改类型的数学perftests从 `once` 到 `loop`. [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([尼古拉\*科切托夫](https://github.com/KochetovNicolai)) -- 添加码头镜像,它允许为我们的代码库构建交互式代码浏览器HTML报告。 [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.com/codebrowser/html_report///ClickHouse/dbms/index.html) +- 添加码头镜像,它允许为我们的代码库构建交互式代码浏览器HTML报告。 [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([阿利沙平](https://github.com/alesapin))见 [Woboq代码浏览器](https://clickhouse.com/codebrowser/ClickHouse/dbms/index.html) - 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) - 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) diff --git a/docs/zh/development/browse-code.md b/docs/zh/development/browse-code.md index 9cee0a37444..f0ad6fd0984 100644 --- a/docs/zh/development/browse-code.md +++ b/docs/zh/development/browse-code.md @@ -5,7 +5,7 @@ toc_title: "\u6D4F\u89C8\u6E90\u4EE3\u7801" # 浏览ClickHouse源代码 {#browse-clickhouse-source-code} -您可以使用 **Woboq** 在线代码浏览器 [点击这里](https://clickhouse.com/codebrowser/html_report/ClickHouse/src/index.html). 它提供了代码导航和语义突出显示、搜索和索引。 代码快照每天更新。 +您可以使用 **Woboq** 在线代码浏览器 [点击这里](https://clickhouse.com/codebrowser/ClickHouse/src/index.html). 它提供了代码导航和语义突出显示、搜索和索引。 代码快照每天更新。 此外,您还可以像往常一样浏览源代码 [GitHub](https://github.com/ClickHouse/ClickHouse) diff --git a/docs/zh/whats-new/changelog/2020.md b/docs/zh/whats-new/changelog/2020.md index 19e9125224c..6890f0f551e 100644 --- a/docs/zh/whats-new/changelog/2020.md +++ b/docs/zh/whats-new/changelog/2020.md @@ -2962,7 +2962,7 @@ * 更新了对 clickhouse-test 脚本中挂起查询的检查. [#8858](https://github.com/ClickHouse/ClickHouse/pull/8858) ([Alexander Kazakov](https://github.com/Akazz)) * 从存储库中删除了一些无用的文件. [#8843](https://github.com/ClickHouse/ClickHouse/pull/8843) ([alexey-milovidov](https://github.com/alexey-milovidov)) * 将数学性能测试的类型从 `once` 更改为 `loop` . [#8783](https://github.com/ClickHouse/ClickHouse/pull/8783) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* 添加 docker 图像,它允许为我们的代码库构建交互式代码浏览器 HTML 报告. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/html_report///ClickHouse/dbms/index.html) +* 添加 docker 图像,它允许为我们的代码库构建交互式代码浏览器 HTML 报告. [#8781](https://github.com/ClickHouse/ClickHouse/pull/8781) ([alesapin](https://github.com/alesapin)) See [Woboq Code Browser](https://clickhouse-test-reports.s3.yandex.net/codebrowser/ClickHouse/dbms/index.html) * 抑制 MSan 下的一些测试失败. [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) * 加速 `exception while insert` 测试. 此测试经常在 debug-with-coverage 构建中超时. [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([alexey-milovidov](https://github.com/alexey-milovidov)) * 将 `libcxx` 和 `libcxxabi` 更新为 master. 准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([alexey-milovidov](https://github.com/alexey-milovidov)) diff --git a/website/sitemap-static.xml b/website/sitemap-static.xml index b5b5f3aa0d5..88888e31b3b 100644 --- a/website/sitemap-static.xml +++ b/website/sitemap-static.xml @@ -17,7 +17,7 @@ weekly - https://clickhouse.com/codebrowser/html_report/ClickHouse/index.html + https://clickhouse.com/codebrowser/ClickHouse/index.html daily From ec3e4251e1d131a4552e52c2b64c528bca964781 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Mar 2022 20:34:25 +0100 Subject: [PATCH 21/32] Fix style --- src/Storages/System/StorageSystemZooKeeper.cpp | 6 ++++-- .../0_stateless/02221_system_zookeeper_unrestricted.sh | 6 +++--- .../02221_system_zookeeper_unrestricted_like.sh | 8 ++++---- 3 files changed, 11 insertions(+), 9 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 120a75ed5c3..87cad40bd40 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -258,7 +258,8 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c paths.pop_front(); String prefix; - if (path_type == ZkPathType::Prefix) { + if (path_type == ZkPathType::Prefix) + { prefix = path; size_t last_slash = prefix.rfind('/'); path = prefix.substr(0, last_slash == String::npos ? 0 : last_slash); @@ -273,7 +274,8 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c if (!prefix.empty()) { // Remove nodes that do not match specified prefix - nodes.erase(std::remove_if(nodes.begin(), nodes.end(), [&prefix, &path_part] (const String & node) { + nodes.erase(std::remove_if(nodes.begin(), nodes.end(), [&prefix, &path_part] (const String & node) + { return (path_part + '/' + node).substr(0, prefix.size()) != prefix; }), nodes.end()); } diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index 92951535157..bd11d7a4429 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -12,7 +12,7 @@ ${CLICKHOUSE_CLIENT} -n -q" CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted', '1') +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted', '1') ORDER BY tuple(); " @@ -20,11 +20,11 @@ ${CLICKHOUSE_CLIENT} -n -q" CREATE TABLE sample_table_2 ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_2', '1') +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_2', '1') ORDER BY tuple(); " -${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query "SELECT name FROM (SELECT path, name FROM system.zookeeper ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted%'"; +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query "SELECT name FROM (SELECT path, name FROM system.zookeeper ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted%'"; ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2" diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh index 8b90d4138d3..7d06ce4d9a3 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -11,7 +11,7 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" ${CLICKHOUSE_CLIENT} -n --query="CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like', '1') +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like', '1') ORDER BY tuple(); DROP TABLE IF EXISTS sample_table;" @@ -19,14 +19,14 @@ DROP TABLE IF EXISTS sample_table;" ${CLICKHOUSE_CLIENT} -n --query "CREATE TABLE sample_table_2 ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/{database}/02221_system_zookeeper_unrestricted_like_2', '1') +ENGINE ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like_2', '1') ORDER BY tuple();" -${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_like%'" +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like%'" ${CLICKHOUSE_CLIENT} --query="SELECT '-------------------------'" -${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse/%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_DATABASE/02221_system_zookeeper_unrestricted_like%'" +${CLICKHOUSE_CLIENT} --allow_unrestricted_reads_from_keeper=1 --query="SELECT name FROM (SELECT path, name FROM system.zookeeper WHERE path LIKE '/clickhouse/%' ORDER BY name) WHERE path LIKE '%$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/02221_system_zookeeper_unrestricted_like%'" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table;" ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS sample_table_2;" From e2989c2b8525e85060b89b6ac45c45a390eefc08 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 1 Mar 2022 21:23:26 +0100 Subject: [PATCH 22/32] Fix storage system zookeeper --- src/Storages/System/StorageSystemZooKeeper.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 87cad40bd40..9c48bea7924 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -194,7 +194,7 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, String pattern = literal->value.safeGet(); bool has_metasymbol = false; - String prefix; // pattern prefix before the first metasymbol occurance + String prefix; // pattern prefix before the first metasymbol occurrence for (size_t i = 0; i < pattern.size(); i++) { char c = pattern[i]; @@ -210,7 +210,7 @@ static bool extractPathImpl(const IAST & elem, Paths & res, ContextPtr context, } } - // Stop prefix on the first metasymbols occurance + // Stop prefix on the first metasymbols occurrence if (c == '_' || c == '%') { has_metasymbol = true; From ec6e58536f96923ed75d140d7ec351ccd681c32a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 1 Mar 2022 21:21:54 +0100 Subject: [PATCH 23/32] Print number of available logical cores in configuration Really just for convenience so the developer can choose appropriate values for CMake options PARALLEL_COMPILE_JOBS and PARALLEL_LINK_JOBS. Might also help to find out why builds are slow. --- cmake/limit_jobs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b85260e6c76..96c6b75bc43 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -55,5 +55,5 @@ endif () if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) message(STATUS "${CMAKE_CURRENT_SOURCE_DIR}: Have ${AVAILABLE_PHYSICAL_MEMORY} megabytes of memory. - Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS}") + Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS} (system has ${NUMBER_OF_LOGICAL_CORES} logical cores)") endif () From 06f47673f40456d28be76464cdd650a9c4dd8d83 Mon Sep 17 00:00:00 2001 From: NikitaEvs Date: Tue, 1 Mar 2022 21:42:27 +0000 Subject: [PATCH 24/32] Fix typo --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexHypothesis.h | 2 +- src/Storages/MergeTree/MergeTreeIndices.h | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 646b672a1e9..965b41f542d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -822,7 +822,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); if (inserted) - it->second.condition = index_helper->createIndexMergedCondtition(query_info, metadata_snapshot); + it->second.condition = index_helper->createIndexMergedCondition(query_info, metadata_snapshot); it->second.addIndex(index_helper); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 30995a162dc..6658730b7c1 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -84,7 +84,7 @@ MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); } -MergeTreeIndexMergedConditionPtr MergeTreeIndexHypothesis::createIndexMergedCondtition( +MergeTreeIndexMergedConditionPtr MergeTreeIndexHypothesis::createIndexMergedCondition( const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const { return std::make_shared( diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index bbdf70a052c..43b56d9559f 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -70,7 +70,7 @@ public: MergeTreeIndexConditionPtr createIndexCondition( const SelectQueryInfo & query, ContextPtr context) const override; - MergeTreeIndexMergedConditionPtr createIndexMergedCondtition( + MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override; bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 1e001d01ada..a761fc3124e 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -164,7 +164,7 @@ struct IMergeTreeIndex virtual MergeTreeIndexConditionPtr createIndexCondition( const SelectQueryInfo & query_info, ContextPtr context) const = 0; - virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondtition( + virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondition( const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, From 6354ce41c121e06bbf4e3633c7f007465f1a8517 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 2 Mar 2022 11:12:42 +0800 Subject: [PATCH 25/32] fix flaky test --- .../0_stateless/01060_window_view_event_tumble_to_asc.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh index e15ac2b8e1a..9163fe8af27 100755 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sh @@ -28,7 +28,7 @@ while true; do $CLICKHOUSE_CLIENT --query="SELECT count(*) FROM dst" | grep -q "7" && break || sleep .5 ||: done -$CLICKHOUSE_CLIENT --query="SELECT * FROM dst ORDER BY w_end;" +$CLICKHOUSE_CLIENT --query="SELECT * FROM dst ORDER BY market, w_end;" $CLICKHOUSE_CLIENT --query="DROP TABLE wv" $CLICKHOUSE_CLIENT --query="DROP TABLE mt" $CLICKHOUSE_CLIENT --query="DROP TABLE dst" From 71c9d4f39cfd00f290c5cf30c69f532fb0bbbb83 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Mar 2022 00:34:09 +0300 Subject: [PATCH 26/32] Fix possible "nmalloc >= ndalloc" jemalloc assertion Fixes: #31531 Refs: https://github.com/jemalloc/jemalloc/pull/2234 Signed-off-by: Azat Khuzhin --- contrib/jemalloc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc b/contrib/jemalloc index ca709c3139f..78b58379c85 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit ca709c3139f77f4c00a903cdee46d71e9028f6c6 +Subproject commit 78b58379c854a639df79beb3289351129d863d4b From 2ea18b308550fae32fa37a2d6c2e59421d156f4a Mon Sep 17 00:00:00 2001 From: HaiBo Li Date: Wed, 2 Mar 2022 15:06:38 +0800 Subject: [PATCH 27/32] Fix the bug that the toDatetime function overflows (#32898) * Fixed overflow bug of the function toDatetime/toDate/toDate32/parseDateTimeBestEffort --- src/Common/DateLUTImpl.h | 8 ++- src/Common/tests/gtest_DateLUTImpl.cpp | 6 ++- src/Functions/FunctionsConversion.h | 50 +++++++++++++++---- src/IO/ReadHelpers.h | 13 ++++- .../integration/test_timezone_config/test.py | 22 ++++++++ .../01186_conversion_to_nullable.reference | 2 +- .../01921_datatype_date32.reference | 4 +- 7 files changed, 89 insertions(+), 16 deletions(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index c178dc58854..5ca37448e36 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1001,8 +1001,12 @@ public: inline LUTIndex makeLUTIndex(Int16 year, UInt8 month, UInt8 day_of_month) const { - if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) + if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) return LUTIndex(0); + + if (unlikely(year > DATE_LUT_MAX_YEAR)) + return LUTIndex(DATE_LUT_SIZE - 1); + auto year_lut_index = (year - DATE_LUT_MIN_YEAR) * 12 + month - 1; UInt32 index = years_months_lut[year_lut_index].toUnderType() + day_of_month - 1; /// When date is out of range, default value is DATE_LUT_SIZE - 1 (2283-11-11) @@ -1012,7 +1016,7 @@ public: /// Create DayNum from year, month, day of month. inline ExtendedDayNum makeDayNum(Int16 year, UInt8 month, UInt8 day_of_month, Int32 default_error_day_num = 0) const { - if (unlikely(year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) + if (unlikely(year < DATE_LUT_MIN_YEAR || month < 1 || month > 12 || day_of_month < 1 || day_of_month > 31)) return ExtendedDayNum(default_error_day_num); return toDayNum(makeLUTIndex(year, month, day_of_month)); diff --git a/src/Common/tests/gtest_DateLUTImpl.cpp b/src/Common/tests/gtest_DateLUTImpl.cpp index 1220c50b409..fd4294fb6ef 100644 --- a/src/Common/tests/gtest_DateLUTImpl.cpp +++ b/src/Common/tests/gtest_DateLUTImpl.cpp @@ -79,8 +79,12 @@ FailuresCount countFailures(const ::testing::TestResult & test_result) TEST(DateLUTTest, makeDayNumTest) { const DateLUTImpl & lut = DateLUT::instance("UTC"); - EXPECT_EQ(0, lut.makeDayNum(2500, 12, 25)); EXPECT_EQ(0, lut.makeDayNum(1924, 12, 31)); + EXPECT_EQ(-1, lut.makeDayNum(1924, 12, 31, -1)); + EXPECT_EQ(-16436, lut.makeDayNum(1925, 1, 1)); + EXPECT_EQ(0, lut.makeDayNum(1970, 1, 1)); + EXPECT_EQ(114635, lut.makeDayNum(2283, 11, 11)); + EXPECT_EQ(114635, lut.makeDayNum(2500, 12, 25)); } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 909803d7cd7..f75d67032f2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -906,6 +906,41 @@ struct ConvertImplGenericToString } }; +/** Conversion of time_t to UInt16, Int32, UInt32 + */ +template +void convertFromTime(typename DataType::FieldType & x, time_t & time) +{ + x = time; +} + +template <> +inline void convertFromTime(DataTypeDate::FieldType & x, time_t & time) +{ + if (unlikely(time < 0)) + x = 0; + else if (unlikely(time > 0xFFFF)) + x = 0xFFFF; + else + x = time; +} + +template <> +inline void convertFromTime(DataTypeDate32::FieldType & x, time_t & time) +{ + x = time; +} + +template <> +inline void convertFromTime(DataTypeDateTime::FieldType & x, time_t & time) +{ + if (unlikely(time < 0)) + x = 0; + else if (unlikely(time > 0xFFFFFFFF)) + x = 0xFFFFFFFF; + else + x = time; +} /** Conversion of strings to numbers, dates, datetimes: through parsing. */ @@ -931,18 +966,16 @@ inline void parseImpl(DataTypeDate32::FieldType & x, ReadBuffer x = tmp; } + // NOTE: no need of extra overload of DateTime64, since readDateTimeText64 has different signature and that case is explicitly handled in the calling code. template <> inline void parseImpl(DataTypeDateTime::FieldType & x, ReadBuffer & rb, const DateLUTImpl * time_zone) { time_t time = 0; readDateTimeText(time, rb, *time_zone); - if (time < 0) - time = 0; - x = time; + convertFromTime(x, time); } - template <> inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { @@ -951,7 +984,6 @@ inline void parseImpl(DataTypeUUID::FieldType & x, ReadBuffer & rb x = tmp.toUnderType(); } - template bool tryParseImpl(typename DataType::FieldType & x, ReadBuffer & rb, const DateLUTImpl *) { @@ -1178,7 +1210,7 @@ struct ConvertThroughParsing { time_t res; parseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; + convertFromTime(vec_to[i], res); } } else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) @@ -1193,7 +1225,7 @@ struct ConvertThroughParsing { time_t res; parseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; + convertFromTime(vec_to[i], res); } } else @@ -1232,14 +1264,14 @@ struct ConvertThroughParsing { time_t res; parsed = tryParseDateTimeBestEffort(res, read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; + convertFromTime(vec_to[i],res); } } else if constexpr (parsing_mode == ConvertFromStringParsingMode::BestEffortUS) { time_t res; parsed = tryParseDateTimeBestEffortUS(res, read_buffer, *local_time_zone, *utc_time_zone); - vec_to[i] = res; + convertFromTime(vec_to[i],res); } else { diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 5d580f6b130..09aec3c3785 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -686,6 +686,16 @@ inline ReturnType readDateTextImpl(LocalDate & date, ReadBuffer & buf) return readDateTextFallback(date, buf); } +inline void convertToDayNum(DayNum & date, ExtendedDayNum & from) +{ + if (unlikely(from < 0)) + date = 0; + else if (unlikely(from > 0xFFFF)) + date = 0xFFFF; + else + date = from; +} + template inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) { @@ -698,7 +708,8 @@ inline ReturnType readDateTextImpl(DayNum & date, ReadBuffer & buf) else if (!readDateTextImpl(local_date, buf)) return false; - date = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day()); + ExtendedDayNum ret = DateLUT::instance().makeDayNum(local_date.year(), local_date.month(), local_date.day()); + convertToDayNum(date,ret); return ReturnType(true); } diff --git a/tests/integration/test_timezone_config/test.py b/tests/integration/test_timezone_config/test.py index ac12eddc709..af7e3548e6a 100644 --- a/tests/integration/test_timezone_config/test.py +++ b/tests/integration/test_timezone_config/test.py @@ -17,3 +17,25 @@ def start_cluster(): def test_check_timezone_config(start_cluster): assert node.query("SELECT toDateTime(1111111111)") == "2005-03-17 17:58:31\n" + +def test_overflow_toDate(start_cluster): + assert node.query("SELECT toDate('2999-12-31','UTC')") == "2149-06-06\n" + assert node.query("SELECT toDate('2021-12-21','UTC')") == "2021-12-21\n" + assert node.query("SELECT toDate('1000-12-31','UTC')") == "1970-01-01\n" + +def test_overflow_toDate32(start_cluster): + assert node.query("SELECT toDate32('2999-12-31','UTC')") == "2283-11-11\n" + assert node.query("SELECT toDate32('2021-12-21','UTC')") == "2021-12-21\n" + assert node.query("SELECT toDate32('1000-12-31','UTC')") == "1925-01-01\n" + +def test_overflow_toDateTime(start_cluster): + assert node.query("SELECT toDateTime('2999-12-31 00:00:00','UTC')") == "2106-02-07 06:28:15\n" + assert node.query("SELECT toDateTime('2106-02-07 06:28:15','UTC')") == "2106-02-07 06:28:15\n" + assert node.query("SELECT toDateTime('1970-01-01 00:00:00','UTC')") == "1970-01-01 00:00:00\n" + assert node.query("SELECT toDateTime('1000-01-01 00:00:00','UTC')") == "1970-01-01 00:00:00\n" + +def test_overflow_parseDateTimeBestEffort(start_cluster): + assert node.query("SELECT parseDateTimeBestEffort('2999-12-31 00:00:00','UTC')") == "2106-02-07 06:28:15\n" + assert node.query("SELECT parseDateTimeBestEffort('2106-02-07 06:28:15','UTC')") == "2106-02-07 06:28:15\n" + assert node.query("SELECT parseDateTimeBestEffort('1970-01-01 00:00:00','UTC')") == "1970-01-01 00:00:00\n" + assert node.query("SELECT parseDateTimeBestEffort('1000-01-01 00:00:00','UTC')") == "1970-01-01 00:00:00\n" diff --git a/tests/queries/0_stateless/01186_conversion_to_nullable.reference b/tests/queries/0_stateless/01186_conversion_to_nullable.reference index 7a690240eb5..dc77029ec3b 100644 --- a/tests/queries/0_stateless/01186_conversion_to_nullable.reference +++ b/tests/queries/0_stateless/01186_conversion_to_nullable.reference @@ -12,7 +12,7 @@ \N 1970-01-01 \N -1970-01-01 +2149-06-06 2020-12-24 01:02:03 \N 1970-01-01 03:00:00 diff --git a/tests/queries/0_stateless/01921_datatype_date32.reference b/tests/queries/0_stateless/01921_datatype_date32.reference index 2114f6f6b1e..8beaefbeb38 100644 --- a/tests/queries/0_stateless/01921_datatype_date32.reference +++ b/tests/queries/0_stateless/01921_datatype_date32.reference @@ -221,13 +221,13 @@ 1925-04-01 1925-04-01 2283-03-31 -1925-01-01 +2283-11-11 2021-09-22 -------addYears--------- 1926-01-01 1926-01-01 2283-11-11 -1925-01-01 +2283-11-11 2022-06-22 -------subtractSeconds--------- 1925-01-01 00:00:00.000 From 1863abe7c56eb6a12cff404096acf2fc4fe27a0d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Mar 2022 12:43:40 +0300 Subject: [PATCH 28/32] disable also for asan --- .../queries/0_stateless/01281_group_by_limit_memory_tracking.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh index ad3fa91e13f..c9c01455e31 100755 --- a/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh +++ b/tests/queries/0_stateless/01281_group_by_limit_memory_tracking.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan +# Tags: no-replicated-database, no-parallel, no-fasttest, no-tsan, no-asan # Tag no-fasttest: max_memory_usage_for_user can interfere another queries running concurrently # Regression for MemoryTracker that had been incorrectly accounted From 9249c5d50e1cd6bbb2810f742c97874f11e7d99d Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 2 Mar 2022 13:09:12 +0300 Subject: [PATCH 29/32] Use tryget instead of get --- src/Storages/System/StorageSystemZooKeeper.cpp | 7 ++++++- .../0_stateless/02221_system_zookeeper_unrestricted.sh | 2 +- .../02221_system_zookeeper_unrestricted_like.sh | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 9c48bea7924..879951df162 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -266,7 +266,12 @@ void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, ContextPtr c } String path_corrected = pathCorrected(path); - zkutil::Strings nodes = zookeeper->getChildren(path_corrected); + + /// Node can be deleted concurrently. It's Ok, we don't provide any + /// consistency guarantees for system.zookeeper table. + zkutil::Strings nodes; + zookeeper->tryGetChildren(path_corrected, nodes); + String path_part = path_corrected; if (path_part == "/") path_part.clear(); diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh index bd11d7a4429..db94c59d2de 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: no-replicated-database, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh index 7d06ce4d9a3..152d8344764 100755 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: no-replicated-database, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From a49106e583db6fa7cd627f36340137dc8462ae60 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Mar 2022 13:21:53 +0300 Subject: [PATCH 30/32] Fix non-MergeTree engines for system.*_log Fixes: https://github.com/ClickHouse/ClickHouse/pull/34949 Signed-off-by: Azat Khuzhin --- src/Interpreters/SystemLog.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index db46097b06a..59545d4314d 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -472,8 +473,13 @@ ASTPtr SystemLog::getCreateTableQuery() /// Write additional (default) settings for MergeTree engine to make it make it possible to compare ASTs /// and recreate tables on settings changes. - auto storage_settings = std::make_unique(getContext()->getMergeTreeSettings()); - storage_settings->loadFromQuery(*create->storage); + const auto & engine = create->storage->engine->as(); + if (endsWith(engine.name, "MergeTree")) + { + auto storage_settings = std::make_unique(getContext()->getMergeTreeSettings()); + storage_settings->loadFromQuery(*create->storage); + } + return create; } From a23ede19e675bd79e13fcc6ab82082b7bccb9a39 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 2 Mar 2022 13:59:55 +0100 Subject: [PATCH 31/32] Update parts.md --- docs/en/operations/system-tables/parts.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 45fdcc40451..cf0f93ecdd6 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -82,7 +82,7 @@ Columns: - `path` ([String](../../sql-reference/data-types/string.md)) – Absolute path to the folder with data part files. -- `disk` ([String](../../sql-reference/data-types/string.md)) – Name of a disk that stores the data part. +- `disk_name` ([String](../../sql-reference/data-types/string.md)) – Name of a disk that stores the data part. - `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of compressed files. From d4cdf04683b67bab9dd18ac249ff48bfe3166054 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 2 Mar 2022 23:23:29 +0800 Subject: [PATCH 32/32] Add missing locks (#34025) --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2e1a11b7ef1..2db93def004 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1056,7 +1056,7 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign } bool scheduled = false; - if (time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred( + if (auto lock = time_after_previous_cleanup_temporary_directories.compareAndRestartDeferred( getSettings()->merge_tree_clear_old_temporary_directories_interval_seconds)) { assignee.scheduleCommonTask(ExecutableLambdaAdapter::create(