From b33a5e0d140dc9c9a942d6bc05c02d292143ae80 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 16 Sep 2020 15:32:33 +0300 Subject: [PATCH 001/131] done --- src/Dictionaries/CacheDictionary.inc.h | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.inc.h b/src/Dictionaries/CacheDictionary.inc.h index 27064d113e6..1f0bca0d01f 100644 --- a/src/Dictionaries/CacheDictionary.inc.h +++ b/src/Dictionaries/CacheDictionary.inc.h @@ -148,15 +148,23 @@ void CacheDictionary::getItemsNumberImpl( std::begin(cache_expired_ids), std::end(cache_expired_ids), std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); - auto on_cell_updated = [&] (const auto id, const auto cell_idx) + auto on_cell_updated = + [&attribute_array, &cache_not_found_ids, &cache_expired_ids, &out] + (const auto id, const auto cell_idx) { const auto attribute_value = attribute_array[cell_idx]; - for (const size_t row : cache_not_found_ids[id]) - out[row] = static_cast(attribute_value); + auto cache_not_found_it = cache_not_found_ids.find(id); - for (const size_t row : cache_expired_ids[id]) - out[row] = static_cast(attribute_value); + if (cache_not_found_it != cache_not_found_ids.end()) + for (const size_t row : cache_not_found_it->second) + out[row] = static_cast(attribute_value); + + auto cache_expired_it = cache_expired_ids.find(id); + + if (cache_expired_it != cache_expired_ids.end()) + for (const size_t row : cache_expired_it->second) + out[row] = static_cast(attribute_value); }; auto on_id_not_found = [&] (auto, auto) {}; From 822a78fa585c9a9e5b342f9d17dc5e0490449e01 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 18 Sep 2020 17:25:45 +0300 Subject: [PATCH 002/131] call under mutex --- src/Dictionaries/CacheDictionary.cpp | 8 ++++---- src/Dictionaries/CacheDictionary.h | 10 ++++++---- src/Dictionaries/CacheDictionary.inc.h | 14 ++++---------- 3 files changed, 14 insertions(+), 18 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 29aee9bfc21..86e8c9390ad 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -908,7 +908,7 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const else cell.setExpiresAt(std::chrono::time_point::max()); - update_unit_ptr->getPresentIdHandler()(id, cell_idx); + update_unit_ptr->callPresentIdHandler(id, cell_idx); /// mark corresponding id as found remaining_ids[id] = 1; } @@ -970,9 +970,9 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const if (was_default) cell.setDefault(); if (was_default) - update_unit_ptr->getAbsentIdHandler()(id, cell_idx); + update_unit_ptr->callAbsentIdHandler(id, cell_idx); else - update_unit_ptr->getPresentIdHandler()(id, cell_idx); + update_unit_ptr->callPresentIdHandler(id, cell_idx); continue; } /// We don't have expired data for that `id` so all we can do is to rethrow `last_exception`. @@ -1004,7 +1004,7 @@ void CacheDictionary::update(UpdateUnitPtr & update_unit_ptr) const setDefaultAttributeValue(attribute, cell_idx); /// inform caller that the cell has not been found - update_unit_ptr->getAbsentIdHandler()(id, cell_idx); + update_unit_ptr->callAbsentIdHandler(id, cell_idx); } ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index 5e7e272ff2e..ee4229b3249 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -399,16 +399,18 @@ private: absent_id_handler([](Key, size_t){}){} - PresentIdHandler getPresentIdHandler() + void callPresentIdHandler(Key key, size_t cell_idx) { std::lock_guard lock(callback_mutex); - return can_use_callback ? present_id_handler : PresentIdHandler{}; + if (can_use_callback) + present_id_handler(key, cell_idx); } - AbsentIdHandler getAbsentIdHandler() + void callAbsentIdHandler(Key key, size_t cell_idx) { std::lock_guard lock(callback_mutex); - return can_use_callback ? absent_id_handler : AbsentIdHandler{}; + if (can_use_callback) + absent_id_handler(key, cell_idx); } std::vector requested_ids; diff --git a/src/Dictionaries/CacheDictionary.inc.h b/src/Dictionaries/CacheDictionary.inc.h index 1f0bca0d01f..5aeed17e2c2 100644 --- a/src/Dictionaries/CacheDictionary.inc.h +++ b/src/Dictionaries/CacheDictionary.inc.h @@ -154,17 +154,11 @@ void CacheDictionary::getItemsNumberImpl( { const auto attribute_value = attribute_array[cell_idx]; - auto cache_not_found_it = cache_not_found_ids.find(id); + for (const size_t row : cache_not_found_ids[id]) + out[row] = static_cast(attribute_value); - if (cache_not_found_it != cache_not_found_ids.end()) - for (const size_t row : cache_not_found_it->second) - out[row] = static_cast(attribute_value); - - auto cache_expired_it = cache_expired_ids.find(id); - - if (cache_expired_it != cache_expired_ids.end()) - for (const size_t row : cache_expired_it->second) - out[row] = static_cast(attribute_value); + for (const size_t row : cache_expired_ids[id]) + out[row] = static_cast(attribute_value); }; auto on_id_not_found = [&] (auto, auto) {}; From 2530b71aa6cd1ede8e53930a6069db0e126f7364 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Fri, 18 Sep 2020 17:48:05 +0300 Subject: [PATCH 003/131] style --- src/Dictionaries/CacheDictionary.inc.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.inc.h b/src/Dictionaries/CacheDictionary.inc.h index 5aeed17e2c2..8867d6a3c4a 100644 --- a/src/Dictionaries/CacheDictionary.inc.h +++ b/src/Dictionaries/CacheDictionary.inc.h @@ -148,9 +148,9 @@ void CacheDictionary::getItemsNumberImpl( std::begin(cache_expired_ids), std::end(cache_expired_ids), std::back_inserter(required_ids), [](auto & pair) { return pair.first; }); - auto on_cell_updated = - [&attribute_array, &cache_not_found_ids, &cache_expired_ids, &out] - (const auto id, const auto cell_idx) + auto on_cell_updated = + [&attribute_array, &cache_not_found_ids, &cache_expired_ids, &out] + (const auto id, const auto cell_idx) { const auto attribute_value = attribute_array[cell_idx]; From 8a8284ce31ce1f53d8d13692f75508b4079d6b8f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:31:49 +0300 Subject: [PATCH 004/131] Avoid deadlocks in Log/TinyLog --- src/Storages/StorageTinyLog.cpp | 73 +++++++++++++++---- src/Storages/StorageTinyLog.h | 2 +- .../0_stateless/01499_log_deadlock.reference | 0 .../0_stateless/01499_log_deadlock.sql | 10 +++ 4 files changed, 68 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/01499_log_deadlock.reference create mode 100644 tests/queries/0_stateless/01499_log_deadlock.sql diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 0bdcab8abf4..1ad8134cdbc 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -46,6 +46,7 @@ namespace DB namespace ErrorCodes { + extern const int TIMEOUT_EXCEEDED; extern const int DUPLICATE_COLUMN; extern const int INCORRECT_FILE_NAME; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -66,10 +67,19 @@ public: return Nested::flatten(res); } - TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_) + TinyLogSource( + size_t block_size_, + const NamesAndTypesList & columns_, + StorageTinyLog & storage_, + std::shared_lock && lock_, + size_t max_read_buffer_size_) : SourceWithProgress(getHeader(columns_)) - , block_size(block_size_), columns(columns_), storage(storage_), lock(storage_.rwlock) - , max_read_buffer_size(max_read_buffer_size_) {} + , block_size(block_size_), columns(columns_), storage(storage_), lock(std::move(lock_)) + , max_read_buffer_size(max_read_buffer_size_) + { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + } String getName() const override { return "TinyLog"; } @@ -80,7 +90,7 @@ private: size_t block_size; NamesAndTypesList columns; StorageTinyLog & storage; - std::shared_lock lock; + std::shared_lock lock; bool is_finished = false; size_t max_read_buffer_size; @@ -110,9 +120,14 @@ private: class TinyLogBlockOutputStream final : public IBlockOutputStream { public: - explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_) - : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock) + explicit TinyLogBlockOutputStream( + StorageTinyLog & storage_, + const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_) + : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(std::move(lock_)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~TinyLogBlockOutputStream() override @@ -140,7 +155,7 @@ public: private: StorageTinyLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -410,7 +425,9 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); + std::unique_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); disk->moveDirectory(table_path, new_path_to_table_data); @@ -424,6 +441,16 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage } +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageTinyLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -437,27 +464,38 @@ Pipe StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. + const Settings & settings = context.getSettingsRef(); + return Pipe(std::make_shared( - max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); + max_block_size, + Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), + *this, + std::shared_lock{rwlock, getLockTimeout(context)}, + settings.max_read_buffer_size)); } -BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, metadata_snapshot); + return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}); } -CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } void StorageTinyLog::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableExclusiveLockHolder &) { - std::unique_lock lock(rwlock); + std::unique_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); disk->clearDirectory(table_path); @@ -470,7 +508,10 @@ void StorageTinyLog::truncate( void StorageTinyLog::drop() { - std::unique_lock lock(rwlock); + std::unique_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + if (disk->exists(table_path)) disk->removeRecursive(table_path); files.clear(); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index dc6ff101503..9c412e4dd14 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -70,7 +70,7 @@ private: Files files; FileChecker file_checker; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; diff --git a/tests/queries/0_stateless/01499_log_deadlock.reference b/tests/queries/0_stateless/01499_log_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01499_log_deadlock.sql b/tests/queries/0_stateless/01499_log_deadlock.sql new file mode 100644 index 00000000000..4765e22b023 --- /dev/null +++ b/tests/queries/0_stateless/01499_log_deadlock.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (x UInt8) ENGINE = TinyLog; + +SET max_execution_time = 1; +INSERT INTO t SELECT * FROM t; -- { serverError 159 } + +SET max_execution_time = 0, lock_acquire_timeout = 1; +INSERT INTO t SELECT * FROM t; -- { serverError 159 } + +DROP TABLE t; From dc7e3b75d3ad3c9c438ac0321a4cbc4c7c2d472a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:40:27 +0300 Subject: [PATCH 005/131] Whitespace --- src/Storages/StorageTinyLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 1ad8134cdbc..84a17e88ed7 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -252,7 +252,7 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum }; if (deserialize_states.count(name) == 0) - type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); } From 3bb0088c86ff56747966fe5048ce6c5d8a0dc538 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:50:25 +0300 Subject: [PATCH 006/131] Whitespace --- src/Storages/StorageTinyLog.cpp | 16 ++++------------ src/Storages/StorageTinyLog.h | 2 +- 2 files changed, 5 insertions(+), 13 deletions(-) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 84a17e88ed7..60ff7de2ea0 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -76,10 +76,10 @@ public: : SourceWithProgress(getHeader(columns_)) , block_size(block_size_), columns(columns_), storage(storage_), lock(std::move(lock_)) , max_read_buffer_size(max_read_buffer_size_) - { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - } + { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + } String getName() const override { return "TinyLog"; } @@ -493,10 +493,6 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context void StorageTinyLog::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableExclusiveLockHolder &) { - std::unique_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - disk->clearDirectory(table_path); files.clear(); @@ -508,10 +504,6 @@ void StorageTinyLog::truncate( void StorageTinyLog::drop() { - std::unique_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - if (disk->exists(table_path)) disk->removeRecursive(table_path); files.clear(); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 9c412e4dd14..41af1e3ff0c 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -70,7 +70,7 @@ private: Files files; FileChecker file_checker; - mutable std::shared_timed_mutex rwlock; + mutable std::mutex mutex; Poco::Logger * log; From 8193da847b2c68e4bb2b0f4674e53da78e556cf4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:51:16 +0300 Subject: [PATCH 007/131] Simplification --- src/Storages/StorageTinyLog.cpp | 2 +- src/Storages/StorageTinyLog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 60ff7de2ea0..cde7d6193f9 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -491,7 +491,7 @@ CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context } void StorageTinyLog::truncate( - const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context & context, TableExclusiveLockHolder &) + const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { disk->clearDirectory(table_path); diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 41af1e3ff0c..9c412e4dd14 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -70,7 +70,7 @@ private: Files files; FileChecker file_checker; - mutable std::mutex mutex; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; From 66ea3c0c43821dd3e57faaf894bca24e146d2ae0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:57:46 +0300 Subject: [PATCH 008/131] Remove useless method --- src/Storages/StorageTinyLog.cpp | 7 ------- src/Storages/StorageTinyLog.h | 2 -- 2 files changed, 9 deletions(-) diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index cde7d6193f9..ab1111c32ca 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -502,13 +502,6 @@ void StorageTinyLog::truncate( addFiles(column.name, *column.type); } -void StorageTinyLog::drop() -{ - if (disk->exists(table_path)) - disk->removeRecursive(table_path); - files.clear(); -} - void registerStorageTinyLog(StorageFactory & factory) { diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 9c412e4dd14..95b7d9f2941 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -43,8 +43,6 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; - void drop() override; - protected: StorageTinyLog( DiskPtr disk_, From bc8fc3e280c0e6afbd3c422c48b0fa775b79036b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Sep 2020 23:59:58 +0300 Subject: [PATCH 009/131] Simplification --- src/Storages/StorageLog.cpp | 4 ---- src/Storages/StorageStripeLog.cpp | 5 ----- 2 files changed, 9 deletions(-) diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index e437bfb05f1..c90611862e3 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -552,8 +552,6 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -569,8 +567,6 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { - std::shared_lock lock(rwlock); - files.clear(); file_count = 0; loaded_marks = false; diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index c4344cf6f1f..21eebeab5e6 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -286,8 +286,6 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -359,10 +357,7 @@ CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Conte void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { - std::shared_lock lock(rwlock); - disk->clearDirectory(table_path); - file_checker = FileChecker{disk, table_path + "sizes.json"}; } From 111acdc63b9146093eb6b527531dc8598387c36c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 00:55:53 +0300 Subject: [PATCH 010/131] Continuation --- src/Storages/StorageFile.cpp | 40 +++++++++---- src/Storages/StorageFile.h | 2 +- src/Storages/StorageLog.cpp | 56 ++++++++++++++----- src/Storages/StorageLog.h | 4 +- src/Storages/StorageStripeLog.cpp | 40 ++++++++++--- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 4 -- .../0_stateless/01499_log_deadlock.reference | 2 + .../0_stateless/01499_log_deadlock.sql | 22 +++++++- 9 files changed, 129 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index cc47047dc78..7b094f9bc06 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,6 +52,7 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int INCORRECT_FILE_NAME; extern const int FILE_DOESNT_EXIST; + extern const int TIMEOUT_EXCEEDED; } namespace @@ -199,6 +200,17 @@ StorageFile::StorageFile(CommonArguments args) setInMemoryMetadata(storage_metadata); } + +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + class StorageFileSource : public SourceWithProgress { public: @@ -245,7 +257,9 @@ public: { if (storage->use_table_fd) { - unique_lock = std::unique_lock(storage->rwlock); + unique_lock = std::unique_lock(storage->rwlock, getLockTimeout(context)); + if (!unique_lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache /// and add ability to seek unseekable files, but cache sync isn't supported. @@ -264,7 +278,9 @@ public: } else { - shared_lock = std::shared_lock(storage->rwlock); + shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context)); + if (!shared_lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } } @@ -373,8 +389,8 @@ private: bool finished_generate = false; - std::shared_lock shared_lock; - std::unique_lock unique_lock; + std::shared_lock shared_lock; + std::unique_lock unique_lock; }; @@ -417,7 +433,7 @@ Pipe StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); return Pipe::unitePipes(std::move(pipes)); } @@ -429,12 +445,16 @@ public: explicit StorageFileBlockOutputStream( StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, + std::unique_lock && lock_, const CompressionMethod compression_method, const Context & context) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + std::unique_ptr naked_buffer = nullptr; if (storage.use_table_fd) { @@ -488,7 +508,7 @@ public: private: StorageFile & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; std::unique_ptr write_buf; BlockOutputStreamPtr writer; bool prefix_written{false}; @@ -506,7 +526,7 @@ BlockOutputStreamPtr StorageFile::write( if (!paths.empty()) path = paths[0]; - return std::make_shared(*this, metadata_snapshot, + return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}, chooseCompressionMethod(path, compression_method), context); } @@ -529,8 +549,6 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (path_new == paths[0]) return; - std::unique_lock lock(rwlock); - Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); @@ -547,8 +565,6 @@ void StorageFile::truncate( if (paths.size() != 1) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); - std::unique_lock lock(rwlock); - if (use_table_fd) { if (0 != ::ftruncate(table_fd, 0)) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index ea70dcd5311..babc56e3a11 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -89,7 +89,7 @@ private: std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log = &Poco::Logger::get("StorageFile"); }; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c90611862e3..d9b37f1ddee 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -39,6 +39,7 @@ namespace DB namespace ErrorCodes { + extern const int TIMEOUT_EXCEEDED; extern const int LOGICAL_ERROR; extern const int DUPLICATE_COLUMN; extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; @@ -50,7 +51,6 @@ namespace ErrorCodes class LogSource final : public SourceWithProgress { public: - static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -116,13 +116,16 @@ private: class LogBlockOutputStream final : public IBlockOutputStream { public: - explicit LogBlockOutputStream(StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + explicit LogBlockOutputStream( + StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) , marks_stream( storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~LogBlockOutputStream() override @@ -149,7 +152,7 @@ public: private: StorageLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -507,9 +510,11 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) } -void StorageLog::loadMarks() +void StorageLog::loadMarks(std::chrono::seconds lock_timeout) { - std::unique_lock lock(rwlock); + std::unique_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); if (loaded_marks) return; @@ -606,6 +611,17 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta return it->second.marks; } + +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -616,11 +632,15 @@ Pipe StorageLog::read( unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - loadMarks(); + + auto lock_timeout = getLockTimeout(context); + loadMarks(lock_timeout); NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); Pipes pipes; @@ -649,18 +669,28 @@ Pipe StorageLog::read( max_read_buffer_size)); } + /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe::unitePipes(std::move(pipes)); } -BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - loadMarks(); - return std::make_shared(*this, metadata_snapshot); + auto lock_timeout = getLockTimeout(context); + loadMarks(lock_timeout); + + std::unique_lock lock(rwlock, lock_timeout); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 49fc9a576c5..3553426b9e6 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -83,7 +83,7 @@ private: DiskPtr disk; String table_path; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Files files; @@ -104,7 +104,7 @@ private: /// Read marks files if they are not already read. /// It is done lazily, so that with a large number of tables, the server starts quickly. /// You can not call with a write locked `rwlock`. - void loadMarks(); + void loadMarks(std::chrono::seconds lock_timeout); /** For normal columns, the number of rows in the block is specified in the marks. * For array columns and nested structures, there are more than one group of marks that correspond to different files diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 21eebeab5e6..4567ac0d452 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -47,13 +47,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; + extern const int TIMEOUT_EXCEEDED; } class StripeLogSource final : public SourceWithProgress { public: - static Block getHeader( StorageStripeLog & storage, const StorageMetadataPtr & metadata_snapshot, @@ -157,10 +157,11 @@ private: class StripeLogBlockOutputStream final : public IBlockOutputStream { public: - explicit StripeLogBlockOutputStream(StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + explicit StripeLogBlockOutputStream( + StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(storage.rwlock) + , lock(std::move(lock_)) , data_out_file(storage.table_path + "data.bin") , data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) , data_out(std::make_unique( @@ -170,6 +171,8 @@ public: , index_out(std::make_unique(*index_out_compressed)) , block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file)) { + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~StripeLogBlockOutputStream() override @@ -223,7 +226,7 @@ public: private: StorageStripeLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; String data_out_file; std::unique_ptr data_out_compressed; @@ -295,6 +298,16 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora } +static std::chrono::seconds getLockTimeout(const Context & context) +{ + const Settings & settings = context.getSettingsRef(); + Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); + if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) + lock_timeout = settings.max_execution_time.totalSeconds(); + return std::chrono::seconds{lock_timeout}; +} + + Pipe StorageStripeLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -304,7 +317,9 @@ Pipe StorageStripeLog::read( const size_t /*max_block_size*/, unsigned num_streams) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); @@ -343,15 +358,22 @@ Pipe StorageStripeLog::read( } -BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) +BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - return std::make_shared(*this, metadata_snapshot); + std::unique_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + return std::make_shared(*this, metadata_snapshot, std::move(lock)); } -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & /* context */) +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock); + std::shared_lock lock(rwlock, getLockTimeout(context)); + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + return file_checker.check(); } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index f88120a932e..ca3bfe4ff75 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -67,7 +67,7 @@ private: size_t max_compress_block_size; FileChecker file_checker; - mutable std::shared_mutex rwlock; + mutable std::shared_timed_mutex rwlock; Poco::Logger * log; }; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index ab1111c32ca..631284bd5cf 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -425,10 +425,6 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage { assert(table_path != new_path_to_table_data); { - std::unique_lock lock(rwlock, std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; diff --git a/tests/queries/0_stateless/01499_log_deadlock.reference b/tests/queries/0_stateless/01499_log_deadlock.reference index e69de29bb2d..91dea2c76e7 100644 --- a/tests/queries/0_stateless/01499_log_deadlock.reference +++ b/tests/queries/0_stateless/01499_log_deadlock.reference @@ -0,0 +1,2 @@ +6 +6 diff --git a/tests/queries/0_stateless/01499_log_deadlock.sql b/tests/queries/0_stateless/01499_log_deadlock.sql index 4765e22b023..700e89a4e4f 100644 --- a/tests/queries/0_stateless/01499_log_deadlock.sql +++ b/tests/queries/0_stateless/01499_log_deadlock.sql @@ -1,10 +1,30 @@ DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt8) ENGINE = TinyLog; -SET max_execution_time = 1; +SET max_execution_time = 1, lock_acquire_timeout = 1000; INSERT INTO t SELECT * FROM t; -- { serverError 159 } SET max_execution_time = 0, lock_acquire_timeout = 1; INSERT INTO t SELECT * FROM t; -- { serverError 159 } DROP TABLE t; + + +SET max_execution_time = 0, lock_acquire_timeout = 1000; + +CREATE TABLE t (x UInt8) ENGINE = Log; + +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; + +DROP TABLE t; + + +CREATE TABLE t (x UInt8) ENGINE = StripeLog; + +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; + +DROP TABLE t; From 4d06bc62b171c0a6e8291007b3ed7ef9af0ed4e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 18 Sep 2020 01:09:27 +0300 Subject: [PATCH 011/131] Allow concurrent reads while writing to TinyLog --- src/Common/FileChecker.cpp | 5 +++ src/Common/FileChecker.h | 4 ++- src/Storages/StorageLog.cpp | 8 ++--- src/Storages/StorageStripeLog.cpp | 6 ++-- src/Storages/StorageTinyLog.cpp | 35 ++++++++++++------- .../0_stateless/01499_log_deadlock.reference | 1 + .../0_stateless/01499_log_deadlock.sql | 10 ++---- 7 files changed, 41 insertions(+), 28 deletions(-) diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index 6cbec3bda77..b306c3af990 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -41,6 +41,11 @@ void FileChecker::setEmpty(const String & full_file_path) map[fileName(full_file_path)] = 0; } +FileChecker::Map FileChecker::getFileSizes() const +{ + return map; +} + CheckResults FileChecker::check() const { // Read the files again every time you call `check` - so as not to violate the constancy. diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 015d4cadb07..59e7331952e 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -27,10 +27,12 @@ public: /// The purpose of this function is to rollback a group of unfinished writes. void repair(); -private: /// File name -> size. using Map = std::map; + Map getFileSizes() const; + +private: void initialize(); void updateImpl(const String & file_path); void load(Map & local_map, const String & path) const; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index d9b37f1ddee..2fbce21655c 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -512,7 +512,7 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) void StorageLog::loadMarks(std::chrono::seconds lock_timeout) { - std::unique_lock lock(rwlock, lock_timeout); + std::unique_lock lock(rwlock, lock_timeout); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -638,7 +638,7 @@ Pipe StorageLog::read( NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); - std::shared_lock lock(rwlock, lock_timeout); + std::shared_lock lock(rwlock, lock_timeout); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -678,7 +678,7 @@ BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMe auto lock_timeout = getLockTimeout(context); loadMarks(lock_timeout); - std::unique_lock lock(rwlock, lock_timeout); + std::unique_lock lock(rwlock, lock_timeout); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -687,7 +687,7 @@ BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMe CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + std::shared_lock lock(rwlock, getLockTimeout(context)); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 4567ac0d452..8ff8035c128 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -317,7 +317,7 @@ Pipe StorageStripeLog::read( const size_t /*max_block_size*/, unsigned num_streams) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + std::shared_lock lock(rwlock, getLockTimeout(context)); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -360,7 +360,7 @@ Pipe StorageStripeLog::read( BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) { - std::unique_lock lock(rwlock, getLockTimeout(context)); + std::unique_lock lock(rwlock, getLockTimeout(context)); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); @@ -370,7 +370,7 @@ BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const Sto CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + std::shared_lock lock(rwlock, getLockTimeout(context)); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 631284bd5cf..4d646c7451e 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -56,7 +57,6 @@ namespace ErrorCodes class TinyLogSource final : public SourceWithProgress { public: - static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -71,14 +71,12 @@ public: size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, - std::shared_lock && lock_, - size_t max_read_buffer_size_) + size_t max_read_buffer_size_, + FileChecker::Map file_sizes_) : SourceWithProgress(getHeader(columns_)) - , block_size(block_size_), columns(columns_), storage(storage_), lock(std::move(lock_)) - , max_read_buffer_size(max_read_buffer_size_) + , block_size(block_size_), columns(columns_), storage(storage_) + , max_read_buffer_size(max_read_buffer_size_), file_sizes(std::move(file_sizes_)) { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } String getName() const override { return "TinyLog"; } @@ -90,19 +88,21 @@ private: size_t block_size; NamesAndTypesList columns; StorageTinyLog & storage; - std::shared_lock lock; bool is_finished = false; size_t max_read_buffer_size; + FileChecker::Map file_sizes; struct Stream { - Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_) + Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_, size_t file_size) : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))), + limited(std::make_unique(*plain, file_size, false)), compressed(*plain) { } std::unique_ptr plain; + std::unique_ptr limited; CompressedReadBuffer compressed; }; @@ -246,7 +246,11 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum String stream_name = IDataType::getFileNameForStream(name, path); if (!streams.count(stream_name)) - streams[stream_name] = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); + { + String file_path = storage.files[stream_name].data_file_path; + streams[stream_name] = std::make_unique( + storage.disk, file_path, max_read_buffer_size, file_sizes[fileName(file_path)]); + } return &streams[stream_name]->compressed; }; @@ -462,12 +466,17 @@ Pipe StorageTinyLog::read( // per column and can't modify it concurrently. const Settings & settings = context.getSettingsRef(); + std::shared_lock lock{rwlock, getLockTimeout(context)}; + if (!lock) + throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + + /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe(std::make_shared( max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, - std::shared_lock{rwlock, getLockTimeout(context)}, - settings.max_read_buffer_size)); + settings.max_read_buffer_size, + file_checker.getFileSizes())); } @@ -479,7 +488,7 @@ BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const Stora CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & context) { - std::shared_lock lock(rwlock, getLockTimeout(context)); + std::shared_lock lock(rwlock, getLockTimeout(context)); if (!lock) throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); diff --git a/tests/queries/0_stateless/01499_log_deadlock.reference b/tests/queries/0_stateless/01499_log_deadlock.reference index 91dea2c76e7..166be640db5 100644 --- a/tests/queries/0_stateless/01499_log_deadlock.reference +++ b/tests/queries/0_stateless/01499_log_deadlock.reference @@ -1,2 +1,3 @@ 6 6 +6 diff --git a/tests/queries/0_stateless/01499_log_deadlock.sql b/tests/queries/0_stateless/01499_log_deadlock.sql index 700e89a4e4f..e98b37f2455 100644 --- a/tests/queries/0_stateless/01499_log_deadlock.sql +++ b/tests/queries/0_stateless/01499_log_deadlock.sql @@ -1,17 +1,13 @@ DROP TABLE IF EXISTS t; CREATE TABLE t (x UInt8) ENGINE = TinyLog; -SET max_execution_time = 1, lock_acquire_timeout = 1000; -INSERT INTO t SELECT * FROM t; -- { serverError 159 } - -SET max_execution_time = 0, lock_acquire_timeout = 1; -INSERT INTO t SELECT * FROM t; -- { serverError 159 } +INSERT INTO t VALUES (1), (2), (3); +INSERT INTO t SELECT * FROM t; +SELECT count() FROM t; DROP TABLE t; -SET max_execution_time = 0, lock_acquire_timeout = 1000; - CREATE TABLE t (x UInt8) ENGINE = Log; INSERT INTO t VALUES (1), (2), (3); From da87861285e63369bd79e176ce375a8d6ea18b85 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 15:15:47 +0300 Subject: [PATCH 012/131] Another test --- .../01502_log_tinylog_deadlock_race.reference | 6 ++ .../01502_log_tinylog_deadlock_race.sh | 85 +++++++++++++++++++ 2 files changed, 91 insertions(+) create mode 100644 tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference create mode 100755 tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference new file mode 100644 index 00000000000..4bf85ae79f3 --- /dev/null +++ b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference @@ -0,0 +1,6 @@ +Testing TinyLog +Done TinyLog +Testing StripeLog +Done StripeLog +Testing Log +Done Log diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh new file mode 100755 index 00000000000..a5b2ff6db8f --- /dev/null +++ b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh @@ -0,0 +1,85 @@ +#!/usr/bin/env bash + +set -e + +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + + +function thread_create { + while true; do + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" + sleep 0.0$RANDOM + done +} + +function thread_drop { + while true; do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" + sleep 0.0$RANDOM + done +} + +function thread_rename { + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done +} + +function thread_select { + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +function thread_insert { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +function thread_insert_select { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done +} + +export -f thread_create +export -f thread_drop +export -f thread_rename +export -f thread_select +export -f thread_insert +export -f thread_insert_select + + +# Do randomized queries and expect nothing extraordinary happens. + +function test_with_engine { + echo "Testing $1" + + timeout 10 bash -c "thread_create t1 $1" & + timeout 10 bash -c "thread_create t2 $1" & + timeout 10 bash -c 'thread_drop t1' & + timeout 10 bash -c 'thread_drop t2' & + timeout 10 bash -c 'thread_rename t1 t2' & + timeout 10 bash -c 'thread_rename t2 t1' & + timeout 10 bash -c 'thread_select t1' & + timeout 10 bash -c 'thread_select t2' & + timeout 10 bash -c 'thread_insert t1 5' & + timeout 10 bash -c 'thread_insert t2 10' & + timeout 10 bash -c 'thread_insert_select t1 t2' & + timeout 10 bash -c 'thread_insert_select t2 t1' & + + wait + echo "Done $1" +} + +test_with_engine TinyLog +test_with_engine StripeLog +test_with_engine Log From 33efb36714e265d5ce5585de27bdf63fd7757294 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 3 Sep 2020 19:52:08 +0300 Subject: [PATCH 013/131] Use previous users.xml if failed to reload. --- src/Access/UsersConfigAccessStorage.cpp | 127 +++++++----------- src/Common/Config/ConfigReloader.cpp | 1 + tests/integration/helpers/test_tools.py | 16 +++ .../configs/config.d/text_log.xml | 3 - .../configs/{users.d => }/custom_settings.xml | 8 -- .../configs/illformed_setting.xml | 7 + .../integration/test_custom_settings/test.py | 32 ++--- .../__init__.py | 0 .../configs/changed_settings.xml | 9 ++ .../configs/normal_settings.xml | 9 ++ .../configs/unexpected_setting_enum.xml | 9 ++ .../configs/unexpected_setting_int.xml | 9 ++ .../configs/unknown_setting.xml | 8 ++ .../test.py | 90 +++++++++++++ 14 files changed, 217 insertions(+), 111 deletions(-) delete mode 100644 tests/integration/test_custom_settings/configs/config.d/text_log.xml rename tests/integration/test_custom_settings/configs/{users.d => }/custom_settings.xml (56%) create mode 100644 tests/integration/test_custom_settings/configs/illformed_setting.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/__init__.py create mode 100644 tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml create mode 100644 tests/integration/test_reloading_settings_from_users_xml/test.py diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 60bcc3784f3..ce10ebf0bcc 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -192,7 +192,7 @@ namespace } - std::vector parseUsers(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + std::vector parseUsers(const Poco::Util::AbstractConfiguration & config) { Poco::Util::AbstractConfiguration::Keys user_names; config.keys("users", user_names); @@ -200,16 +200,8 @@ namespace std::vector users; users.reserve(user_names.size()); for (const auto & user_name : user_names) - { - try - { - users.push_back(parseUser(config, user_name)); - } - catch (...) - { - tryLogCurrentException(log, "Could not parse user " + backQuote(user_name)); - } - } + users.push_back(parseUser(config, user_name)); + return users; } @@ -256,12 +248,11 @@ namespace } quota->to_roles.add(user_ids); - return quota; } - std::vector parseQuotas(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + std::vector parseQuotas(const Poco::Util::AbstractConfiguration & config) { Poco::Util::AbstractConfiguration::Keys user_names; config.keys("users", user_names); @@ -278,76 +269,63 @@ namespace quotas.reserve(quota_names.size()); for (const auto & quota_name : quota_names) { - try - { - auto it = quota_to_user_ids.find(quota_name); - const std::vector & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector{}; - quotas.push_back(parseQuota(config, quota_name, quota_users)); - } - catch (...) - { - tryLogCurrentException(log, "Could not parse quota " + backQuote(quota_name)); - } + auto it = quota_to_user_ids.find(quota_name); + const std::vector & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector{}; + quotas.push_back(parseQuota(config, quota_name, quota_users)); } return quotas; } - std::vector parseRowPolicies(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + std::vector parseRowPolicies(const Poco::Util::AbstractConfiguration & config) { std::map, std::unordered_map> all_filters_map; + Poco::Util::AbstractConfiguration::Keys user_names; + config.keys("users", user_names); - try + for (const String & user_name : user_names) { - config.keys("users", user_names); - for (const String & user_name : user_names) + const String databases_config = "users." + user_name + ".databases"; + if (config.has(databases_config)) { - const String databases_config = "users." + user_name + ".databases"; - if (config.has(databases_config)) + Poco::Util::AbstractConfiguration::Keys database_keys; + config.keys(databases_config, database_keys); + + /// Read tables within databases + for (const String & database_key : database_keys) { - Poco::Util::AbstractConfiguration::Keys database_keys; - config.keys(databases_config, database_keys); + const String database_config = databases_config + "." + database_key; - /// Read tables within databases - for (const String & database_key : database_keys) + String database_name; + if (((database_key == "database") || (database_key.starts_with("database["))) && config.has(database_config + "[@name]")) + database_name = config.getString(database_config + "[@name]"); + else if (size_t bracket_pos = database_key.find('['); bracket_pos != std::string::npos) + database_name = database_key.substr(0, bracket_pos); + else + database_name = database_key; + + Poco::Util::AbstractConfiguration::Keys table_keys; + config.keys(database_config, table_keys); + + /// Read table properties + for (const String & table_key : table_keys) { - const String database_config = databases_config + "." + database_key; - - String database_name; - if (((database_key == "database") || (database_key.starts_with("database["))) && config.has(database_config + "[@name]")) - database_name = config.getString(database_config + "[@name]"); - else if (size_t bracket_pos = database_key.find('['); bracket_pos != std::string::npos) - database_name = database_key.substr(0, bracket_pos); + String table_config = database_config + "." + table_key; + String table_name; + if (((table_key == "table") || (table_key.starts_with("table["))) && config.has(table_config + "[@name]")) + table_name = config.getString(table_config + "[@name]"); + else if (size_t bracket_pos = table_key.find('['); bracket_pos != std::string::npos) + table_name = table_key.substr(0, bracket_pos); else - database_name = database_key; + table_name = table_key; - Poco::Util::AbstractConfiguration::Keys table_keys; - config.keys(database_config, table_keys); - - /// Read table properties - for (const String & table_key : table_keys) - { - String table_config = database_config + "." + table_key; - String table_name; - if (((table_key == "table") || (table_key.starts_with("table["))) && config.has(table_config + "[@name]")) - table_name = config.getString(table_config + "[@name]"); - else if (size_t bracket_pos = table_key.find('['); bracket_pos != std::string::npos) - table_name = table_key.substr(0, bracket_pos); - else - table_name = table_key; - - String filter_config = table_config + ".filter"; - all_filters_map[{database_name, table_name}][user_name] = config.getString(filter_config); - } + String filter_config = table_config + ".filter"; + all_filters_map[{database_name, table_name}][user_name] = config.getString(filter_config); } } } } - catch (...) - { - tryLogCurrentException(log, "Could not parse row policies"); - } std::vector policies; for (auto & [database_and_table_name, user_to_filters] : all_filters_map) @@ -450,23 +428,14 @@ namespace std::vector parseSettingsProfiles( const Poco::Util::AbstractConfiguration & config, - const std::function & check_setting_name_function, - Poco::Logger * log) + const std::function & check_setting_name_function) { std::vector profiles; Poco::Util::AbstractConfiguration::Keys profile_names; config.keys("profiles", profile_names); for (const auto & profile_name : profile_names) - { - try - { - profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function)); - } - catch (...) - { - tryLogCurrentException(log, "Could not parse profile " + backQuote(profile_name)); - } - } + profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function)); + return profiles; } } @@ -520,13 +489,13 @@ void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config) { std::vector> all_entities; - for (const auto & entity : parseUsers(config, getLogger())) + for (const auto & entity : parseUsers(config)) all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseQuotas(config, getLogger())) + for (const auto & entity : parseQuotas(config)) all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseRowPolicies(config, getLogger())) + for (const auto & entity : parseRowPolicies(config)) all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function, getLogger())) + for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function)) all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } diff --git a/src/Common/Config/ConfigReloader.cpp b/src/Common/Config/ConfigReloader.cpp index d4a2dfbafe5..677448e03ae 100644 --- a/src/Common/Config/ConfigReloader.cpp +++ b/src/Common/Config/ConfigReloader.cpp @@ -138,6 +138,7 @@ void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallbac if (throw_on_error) throw; tryLogCurrentException(log, "Error updating configuration from '" + path + "' config."); + return; } LOG_DEBUG(log, "Loaded config '{}', performed update on configuration", path); diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index d196142c518..9fbffe41819 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -60,3 +60,19 @@ def assert_eq_with_retry(instance, query, expectation, retry_count=20, sleep_tim if expectation_tsv != val: raise AssertionError("'{}' != '{}'\n{}".format(expectation_tsv, val, '\n'.join( expectation_tsv.diff(val, n1="expectation", n2="query")))) + +def assert_logs_contain(instance, substring): + if not instance.contains_in_log(substring): + raise AssertionError("'{}' not found in logs".format(substring)) + +def assert_logs_contain_with_retry(instance, substring, retry_count=20, sleep_time=0.5): + for i in xrange(retry_count): + try: + if instance.contains_in_log(substring): + break + time.sleep(sleep_time) + except Exception as ex: + print "contains_in_log_with_retry retry {} exception {}".format(i + 1, ex) + time.sleep(sleep_time) + else: + raise AssertionError("'{}' not found in logs".format(substring)) diff --git a/tests/integration/test_custom_settings/configs/config.d/text_log.xml b/tests/integration/test_custom_settings/configs/config.d/text_log.xml deleted file mode 100644 index f386249f170..00000000000 --- a/tests/integration/test_custom_settings/configs/config.d/text_log.xml +++ /dev/null @@ -1,3 +0,0 @@ - - - diff --git a/tests/integration/test_custom_settings/configs/users.d/custom_settings.xml b/tests/integration/test_custom_settings/configs/custom_settings.xml similarity index 56% rename from tests/integration/test_custom_settings/configs/users.d/custom_settings.xml rename to tests/integration/test_custom_settings/configs/custom_settings.xml index f32d0f3626d..d3865b434e6 100644 --- a/tests/integration/test_custom_settings/configs/users.d/custom_settings.xml +++ b/tests/integration/test_custom_settings/configs/custom_settings.xml @@ -6,13 +6,5 @@ Float64_-43.25e-1 'some text' - - - 1 - - - - 1 - diff --git a/tests/integration/test_custom_settings/configs/illformed_setting.xml b/tests/integration/test_custom_settings/configs/illformed_setting.xml new file mode 100644 index 00000000000..267978a8af9 --- /dev/null +++ b/tests/integration/test_custom_settings/configs/illformed_setting.xml @@ -0,0 +1,7 @@ + + + + 1 + + + diff --git a/tests/integration/test_custom_settings/test.py b/tests/integration/test_custom_settings/test.py index 32df79ec1e9..7e147f999a9 100644 --- a/tests/integration/test_custom_settings/test.py +++ b/tests/integration/test_custom_settings/test.py @@ -1,9 +1,10 @@ import pytest +import os from helpers.cluster import ClickHouseCluster +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) -node = cluster.add_instance('node', main_configs=["configs/config.d/text_log.xml"], - user_configs=["configs/users.d/custom_settings.xml"]) +node = cluster.add_instance('node') @pytest.fixture(scope="module", autouse=True) @@ -16,28 +17,17 @@ def started_cluster(): cluster.shutdown() -def test(): +def test_custom_settings(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/custom_settings.xml"), '/etc/clickhouse-server/users.d/z.xml') + node.query("SYSTEM RELOAD CONFIG") + assert node.query("SELECT getSetting('custom_a')") == "-5\n" assert node.query("SELECT getSetting('custom_b')") == "10000000000\n" assert node.query("SELECT getSetting('custom_c')") == "-4.325\n" assert node.query("SELECT getSetting('custom_d')") == "some text\n" - assert "custom_a = -5, custom_b = 10000000000, custom_c = -4.325, custom_d = \\'some text\\'" \ - in node.query("SHOW CREATE SETTINGS PROFILE default") - assert "no settings profile" in node.query_and_get_error( - "SHOW CREATE SETTINGS PROFILE profile_with_unknown_setting") - assert "no settings profile" in node.query_and_get_error("SHOW CREATE SETTINGS PROFILE profile_illformed_setting") - - -def test_invalid_settings(): - node.query("SYSTEM RELOAD CONFIG") - node.query("SYSTEM FLUSH LOGS") - - assert node.query("SELECT COUNT() FROM system.text_log WHERE" - " message LIKE '%Could not parse profile `profile_illformed_setting`%'" - " AND message LIKE '%Couldn\\'t restore Field from dump%'") == "1\n" - - assert node.query("SELECT COUNT() FROM system.text_log WHERE" - " message LIKE '%Could not parse profile `profile_with_unknown_setting`%'" - " AND message LIKE '%Setting x is neither a builtin setting nor started with the prefix \\'custom_\\'%'") == "1\n" +def test_illformed_setting(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/illformed_setting.xml"), '/etc/clickhouse-server/users.d/z.xml') + error_message = "Couldn't restore Field from dump: 1" + assert error_message in node.query_and_get_error("SYSTEM RELOAD CONFIG") diff --git a/tests/integration/test_reloading_settings_from_users_xml/__init__.py b/tests/integration/test_reloading_settings_from_users_xml/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml new file mode 100644 index 00000000000..382c2b2dc20 --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/changed_settings.xml @@ -0,0 +1,9 @@ + + + + + 20000000000 + nearest_hostname + + + diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml new file mode 100644 index 00000000000..85d1c26659f --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/normal_settings.xml @@ -0,0 +1,9 @@ + + + + + 10000000000 + first_or_random + + + diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml new file mode 100644 index 00000000000..ff2b40583de --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_enum.xml @@ -0,0 +1,9 @@ + + + + + 20000000000 + a + + + diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml new file mode 100644 index 00000000000..4ef15ed3680 --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unexpected_setting_int.xml @@ -0,0 +1,9 @@ + + + + + a + nearest_hostname + + + diff --git a/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml b/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml new file mode 100644 index 00000000000..9bac09aef18 --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/configs/unknown_setting.xml @@ -0,0 +1,8 @@ + + + + + 8 + + + diff --git a/tests/integration/test_reloading_settings_from_users_xml/test.py b/tests/integration/test_reloading_settings_from_users_xml/test.py new file mode 100644 index 00000000000..b45568ee904 --- /dev/null +++ b/tests/integration/test_reloading_settings_from_users_xml/test.py @@ -0,0 +1,90 @@ +import pytest +import os +import time +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain_with_retry + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', user_configs=["configs/normal_settings.xml"]) + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_to_normal_settings_after_test(): + try: + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/normal_settings.xml"), '/etc/clickhouse-server/users.d/z.xml') + node.query("SYSTEM RELOAD CONFIG") + yield + finally: + pass + + +def test_force_reload(): + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" + + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/changed_settings.xml"), '/etc/clickhouse-server/users.d/z.xml') + node.query("SYSTEM RELOAD CONFIG") + + assert node.query("SELECT getSetting('max_memory_usage')") == "20000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "nearest_hostname\n" + + +def test_reload_on_timeout(): + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" + + time.sleep(1) # The modification time of the 'z.xml' file should be different, + # because config files are reload by timer only when the modification time is changed. + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/changed_settings.xml"), '/etc/clickhouse-server/users.d/z.xml') + + assert_eq_with_retry(node, "SELECT getSetting('max_memory_usage')", "20000000000") + assert_eq_with_retry(node, "SELECT getSetting('load_balancing')", "nearest_hostname") + + +def test_unknown_setting_force_reload(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/unknown_setting.xml"), '/etc/clickhouse-server/users.d/z.xml') + + error_message = "Setting xyz is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings" + assert error_message in node.query_and_get_error("SYSTEM RELOAD CONFIG") + + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" + + +def test_unknown_setting_reload_on_timeout(): + time.sleep(1) # The modification time of the 'z.xml' file should be different, + # because config files are reload by timer only when the modification time is changed. + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/unknown_setting.xml"), '/etc/clickhouse-server/users.d/z.xml') + + error_message = "Setting xyz is neither a builtin setting nor started with the prefix 'custom_' registered for user-defined settings" + assert_logs_contain_with_retry(node, error_message) + + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" + + +def test_unexpected_setting_int(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/unexpected_setting_int.xml"), '/etc/clickhouse-server/users.d/z.xml') + error_message = "Cannot parse" + assert error_message in node.query_and_get_error("SYSTEM RELOAD CONFIG") + + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" + + +def test_unexpected_setting_enum(): + node.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/unexpected_setting_int.xml"), '/etc/clickhouse-server/users.d/z.xml') + error_message = "Cannot parse" + assert error_message in node.query_and_get_error("SYSTEM RELOAD CONFIG") + + assert node.query("SELECT getSetting('max_memory_usage')") == "10000000000\n" + assert node.query("SELECT getSetting('load_balancing')") == "first_or_random\n" From 194c8da69a302f5cff694ea22d84cacc1db53fa3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 19 Sep 2020 19:49:24 +0300 Subject: [PATCH 014/131] Tweak test --- ...ference => 01502_long_log_tinylog_deadlock_race.reference} | 0 ...adlock_race.sh => 01502_long_log_tinylog_deadlock_race.sh} | 4 ++-- 2 files changed, 2 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/{01502_log_tinylog_deadlock_race.reference => 01502_long_log_tinylog_deadlock_race.reference} (100%) rename tests/queries/0_stateless/{01502_log_tinylog_deadlock_race.sh => 01502_long_log_tinylog_deadlock_race.sh} (89%) diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference similarity index 100% rename from tests/queries/0_stateless/01502_log_tinylog_deadlock_race.reference rename to tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh similarity index 89% rename from tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh rename to tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh index a5b2ff6db8f..29c5f868617 100755 --- a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -10,14 +10,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function thread_create { while true; do - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' sleep 0.0$RANDOM done } function thread_drop { while true; do - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' sleep 0.0$RANDOM done } From a28867486792552775855d16404b1596c99e9d19 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 19 Sep 2020 17:20:49 +0300 Subject: [PATCH 015/131] Fix testflows tests. --- docker/test/testflows/runner/Dockerfile | 2 +- tests/testflows/ldap/tests/user_config.py | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docker/test/testflows/runner/Dockerfile b/docker/test/testflows/runner/Dockerfile index 898552ade56..ed49743319c 100644 --- a/docker/test/testflows/runner/Dockerfile +++ b/docker/test/testflows/runner/Dockerfile @@ -35,7 +35,7 @@ RUN apt-get update \ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone -RUN pip3 install urllib3 testflows==1.6.42 docker-compose docker dicttoxml kazoo tzlocal +RUN pip3 install urllib3 testflows==1.6.48 docker-compose docker dicttoxml kazoo tzlocal ENV DOCKER_CHANNEL stable ENV DOCKER_VERSION 17.09.1-ce diff --git a/tests/testflows/ldap/tests/user_config.py b/tests/testflows/ldap/tests/user_config.py index edc85a5877e..f609231b752 100644 --- a/tests/testflows/ldap/tests/user_config.py +++ b/tests/testflows/ldap/tests/user_config.py @@ -29,6 +29,7 @@ def empty_user_name(self, timeout=20): def empty_server_name(self, timeout=20): """Check that if server name is an empty string then login is not allowed. """ + message = "Exception: LDAP server name cannot be empty for user" servers = {"openldap1": { "host": "openldap1", "port": "389", "enable_tls": "no", "auth_dn_prefix": "cn=", "auth_dn_suffix": ",ou=users,dc=company,dc=com" @@ -37,7 +38,8 @@ def empty_server_name(self, timeout=20): "errorcode": 4, "message": "DB::Exception: user1: Authentication failed: password is incorrect or there is no user with such name" }] - login(servers, *users) + config = create_ldap_users_config_content(*users) + invalid_user_config(servers, config, message=message, tail=15, timeout=timeout) @TestScenario @Requirements( @@ -147,9 +149,6 @@ def ldap_and_password(self): with Then("I expect an error when I try to load the configuration file", description=error_message): invalid_user_config(servers, new_config, message=error_message, tail=16) - with And("I expect the authentication to fail when I try to login"): - login(servers, user, config=new_config) - @TestFeature @Name("user config") def feature(self, node="clickhouse1"): From 25a1f8263113622f375caa45a1ab3beaf451e70d Mon Sep 17 00:00:00 2001 From: Mark Papadakis Date: Sun, 20 Sep 2020 20:26:44 +0300 Subject: [PATCH 016/131] Update ForkProcessor.cpp Got rid of an unnecessary local --- src/Processors/ForkProcessor.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Processors/ForkProcessor.cpp b/src/Processors/ForkProcessor.cpp index 7fa21c4236d..fe836ca740d 100644 --- a/src/Processors/ForkProcessor.cpp +++ b/src/Processors/ForkProcessor.cpp @@ -10,7 +10,6 @@ ForkProcessor::Status ForkProcessor::prepare() /// Check can output. - bool all_finished = true; bool all_can_push = true; size_t num_active_outputs = 0; @@ -18,7 +17,6 @@ ForkProcessor::Status ForkProcessor::prepare() { if (!output.isFinished()) { - all_finished = false; ++num_active_outputs; /// The order is important. @@ -27,8 +25,7 @@ ForkProcessor::Status ForkProcessor::prepare() } } - if (all_finished) - { + if (0 == num_active_outputs) { input.close(); return Status::Finished; } From ffe4f4b3668cd778216ef0ee098a773e71bb9edb Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 20 Sep 2020 22:59:26 +0300 Subject: [PATCH 017/131] Update ForkProcessor.cpp --- src/Processors/ForkProcessor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/ForkProcessor.cpp b/src/Processors/ForkProcessor.cpp index fe836ca740d..9b17f8ad5ca 100644 --- a/src/Processors/ForkProcessor.cpp +++ b/src/Processors/ForkProcessor.cpp @@ -25,7 +25,8 @@ ForkProcessor::Status ForkProcessor::prepare() } } - if (0 == num_active_outputs) { + if (0 == num_active_outputs) + { input.close(); return Status::Finished; } From 13e41cdd20c8204efedcf9c7219443a16915c1e0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 21 Sep 2020 13:24:10 +0300 Subject: [PATCH 018/131] try enable atomic database by default --- docker/test/fasttest/run.sh | 1 + docker/test/stateful/run.sh | 7 +++--- docker/test/stateless/run.sh | 7 +++--- docker/test/stateless_unbundled/run.sh | 7 +++--- docker/test/stress/stress | 2 +- programs/copier/Internals.cpp | 17 +++----------- programs/copier/Internals.h | 2 +- programs/copier/TaskTableAndShard.h | 22 +++++++++++-------- src/Core/Settings.h | 2 +- src/Storages/StorageDictionary.cpp | 8 +++++-- src/Storages/StorageDictionary.h | 3 ++- tests/clickhouse-test | 8 +++---- tests/config/database_atomic_usersd.xml | 1 - tests/config/database_ordinary_usersd.xml | 7 ++++++ .../test_cluster_copier/task0_description.xml | 4 ++-- .../task_month_to_week_description.xml | 4 ++-- .../task_test_block_size.xml | 4 ++-- tests/integration/test_cluster_copier/test.py | 21 ++++++++++-------- .../test_cluster_copier/trivial_test.py | 2 +- .../test_dictionaries_dependency/test.py | 2 +- ...lized_view_and_deduplication_zookeeper.sql | 16 +++++++------- .../00604_show_create_database.reference | 2 +- .../0_stateless/00609_mv_index_in_in.sql | 4 ++-- .../0_stateless/00738_lock_for_inner_table.sh | 4 ++-- .../01190_full_attach_syntax.reference | 18 +++++++-------- .../0_stateless/01190_full_attach_syntax.sql | 16 +++----------- .../01224_no_superfluous_dict_reload.sql | 2 +- ...1225_show_create_table_from_dictionary.sql | 2 +- ...9_bad_arguments_for_bloom_filter.reference | 6 ++--- .../01249_bad_arguments_for_bloom_filter.sql | 6 ++++- ...20_create_sync_race_condition_zookeeper.sh | 9 ++++---- tests/queries/skip_list.json | 16 +++++++------- 32 files changed, 119 insertions(+), 113 deletions(-) create mode 100644 tests/config/database_ordinary_usersd.xml diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index ccbadb84f27..560fa9c77af 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -136,6 +136,7 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml # Keep original query_masking_rules.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c3576acc0e4..34980508488 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -23,10 +23,11 @@ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-serv ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ fi function start() diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 4a9ad891883..5546f0eef18 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -33,6 +33,8 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ # Retain any pre-existing config and allow ClickHouse to load it if required ln -s --backup=simple --suffix=_original.xml \ @@ -41,9 +43,8 @@ ln -s --backup=simple --suffix=_original.xml \ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_usersd.xml /etc/clickhouse-server/users.d/ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 4a9ad891883..13ded2290fb 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -33,6 +33,8 @@ ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/conf ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ +ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ +ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ # Retain any pre-existing config and allow ClickHouse to load it if required ln -s --backup=simple --suffix=_original.xml \ @@ -41,9 +43,8 @@ ln -s --backup=simple --suffix=_original.xml \ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY + ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 60db5ec465c..a36adda3aad 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -29,7 +29,7 @@ def get_options(i): if 0 < i: options += " --order=random" if i % 2 == 1: - options += " --atomic-db-engine" + options += " --db-engine=Ordinary" return options diff --git a/programs/copier/Internals.cpp b/programs/copier/Internals.cpp index 12da07a772a..ca26f0d1831 100644 --- a/programs/copier/Internals.cpp +++ b/programs/copier/Internals.cpp @@ -215,31 +215,20 @@ Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast) return primary_key_columns; } -String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast) +bool isReplicatedTableEngine(const ASTPtr & storage_ast) { - String storage_str = queryToString(storage_ast); - const auto & storage = storage_ast->as(); const auto & engine = storage.engine->as(); if (!endsWith(engine.name, "MergeTree")) { + String storage_str = queryToString(storage_ast); throw Exception( "Unsupported engine was specified in " + storage_str + ", only *MergeTree engines are supported", ErrorCodes::BAD_ARGUMENTS); } - if (!startsWith(engine.name, "Replicated")) - { - return ""; - } - - auto replicated_table_arguments = engine.arguments->children; - - auto zk_table_path_ast = replicated_table_arguments[0]->as(); - auto zk_table_path_string = zk_table_path_ast.value.safeGet(); - - return zk_table_path_string; + return startsWith(engine.name, "Replicated"); } ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random) diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h index b61b6d59629..7e45c0ea2ee 100644 --- a/programs/copier/Internals.h +++ b/programs/copier/Internals.h @@ -200,7 +200,7 @@ ASTPtr extractOrderBy(const ASTPtr & storage_ast); Names extractPrimaryKeyColumnNames(const ASTPtr & storage_ast); -String extractReplicatedTableZookeeperPath(const ASTPtr & storage_ast); +bool isReplicatedTableEngine(const ASTPtr & storage_ast); ShardPriority getReplicasPriority(const Cluster::Addresses & replicas, const std::string & local_hostname, UInt8 random); diff --git a/programs/copier/TaskTableAndShard.h b/programs/copier/TaskTableAndShard.h index 11ceffd12cd..27c4b89377d 100644 --- a/programs/copier/TaskTableAndShard.h +++ b/programs/copier/TaskTableAndShard.h @@ -48,7 +48,7 @@ struct TaskTable String getCertainPartitionPieceTaskStatusPath(const String & partition_name, const size_t piece_number) const; - bool isReplicatedTable() const { return engine_push_zk_path != ""; } + bool isReplicatedTable() const { return is_replicated_table; } /// Partitions will be split into number-of-splits pieces. /// Each piece will be copied independently. (10 by default) @@ -78,6 +78,7 @@ struct TaskTable /// First argument of Replicated...MergeTree() String engine_push_zk_path; + bool is_replicated_table; ASTPtr rewriteReplicatedCreateQueryToPlain(); @@ -269,7 +270,7 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); primary_key_comma_separated = Nested::createCommaSeparatedStringFrom(extractPrimaryKeyColumnNames(engine_push_ast)); - engine_push_zk_path = extractReplicatedTableZookeeperPath(engine_push_ast); + is_replicated_table = isReplicatedTableEngine(engine_push_ast); } sharding_key_str = config.getString(table_prefix + "sharding_key"); @@ -372,15 +373,18 @@ inline ASTPtr TaskTable::rewriteReplicatedCreateQueryToPlain() auto & new_storage_ast = prev_engine_push_ast->as(); auto & new_engine_ast = new_storage_ast.engine->as(); - auto & replicated_table_arguments = new_engine_ast.arguments->children; - - /// Delete first two arguments of Replicated...MergeTree() table. - replicated_table_arguments.erase(replicated_table_arguments.begin()); - replicated_table_arguments.erase(replicated_table_arguments.begin()); - - /// Remove replicated from name + /// Remove "Replicated" from name new_engine_ast.name = new_engine_ast.name.substr(10); + if (new_engine_ast.arguments) + { + auto & replicated_table_arguments = new_engine_ast.arguments->children; + + /// Delete first two arguments of Replicated...MergeTree() table. + replicated_table_arguments.erase(replicated_table_arguments.begin()); + replicated_table_arguments.erase(replicated_table_arguments.begin()); + } + return new_storage_ast.clone(); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b96b1b12c24..9449cd571a1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -350,7 +350,7 @@ class IColumn; M(UInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(UInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ - M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Ordinary, "Default database engine.", 0) \ + M(DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic, "Default database engine.", 0) \ M(Bool, show_table_uuid_in_table_create_query_if_not_nil, false, "For tables in databases with Engine=Atomic show UUID of the table in its CREATE query.", 0) \ M(Bool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ M(Bool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 5d92b9cec55..99645d09d00 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -100,6 +100,9 @@ StorageDictionary::StorageDictionary( Location location_) : IStorage(table_id_) , dictionary_name(dictionary_name_) + , resolved_dictionary_name(location_ == Location::SameDatabaseAndNameAsDictionary + ? dictionary_name + : DatabaseCatalog::instance().resolveDictionaryName(dictionary_name)) , location(location_) { StorageInMemoryMetadata storage_metadata; @@ -132,7 +135,7 @@ Pipe StorageDictionary::read( const size_t max_block_size, const unsigned /*threads*/) { - auto dictionary = context.getExternalDictionariesLoader().getDictionary(dictionary_name); + auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolved_dictionary_name); auto stream = dictionary->getBlockInputStream(column_names, max_block_size); /// TODO: update dictionary interface for processors. return Pipe(std::make_shared(stream)); @@ -152,7 +155,8 @@ void registerStorageDictionary(StorageFactory & factory) if (!args.attach) { - const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(dictionary_name); + auto resolved = DatabaseCatalog::instance().resolveDictionaryName(dictionary_name); + const auto & dictionary = args.context.getExternalDictionariesLoader().getDictionary(resolved); const DictionaryStructure & dictionary_structure = dictionary->getStructure(); checkNamesAndTypesCompatibleWithDictionary(dictionary_name, args.columns, dictionary_structure); } diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index d822552124d..528ee6533b7 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -28,7 +28,7 @@ public: static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static String generateNamesAndTypesDescription(const NamesAndTypesList & list); - const String & dictionaryName() const { return dictionary_name; } + const String & dictionaryName() const { return resolved_dictionary_name; } /// Specifies where the table is located relative to the dictionary. enum class Location @@ -50,6 +50,7 @@ public: private: const String dictionary_name; + const String resolved_dictionary_name; const Location location; protected: diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3bed189d55..de296cabb7a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,9 +107,9 @@ def remove_control_characters(s): return s def get_db_engine(args): - if args.atomic_db_engine: - return " ENGINE=Atomic" - return "" + if args.db_engine: + return " ENGINE=" + args.db_engine + return "" # Will use default engine def run_single_test(args, ext, server_logs_level, client_options, case_file, stdout_file, stderr_file): @@ -792,7 +792,7 @@ if __name__ == '__main__': parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started') parser.add_argument('--skip-list-path', help="Path to skip-list file") parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found") - parser.add_argument('--atomic-db-engine', action='store_true', help='Create databases with Atomic engine by default') + parser.add_argument('--db-engine', help='Database engine name') parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests') parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') diff --git a/tests/config/database_atomic_usersd.xml b/tests/config/database_atomic_usersd.xml index 201d476da24..58e36cf3884 100644 --- a/tests/config/database_atomic_usersd.xml +++ b/tests/config/database_atomic_usersd.xml @@ -1,7 +1,6 @@ - Atomic 0 diff --git a/tests/config/database_ordinary_usersd.xml b/tests/config/database_ordinary_usersd.xml new file mode 100644 index 00000000000..68f3b044f75 --- /dev/null +++ b/tests/config/database_ordinary_usersd.xml @@ -0,0 +1,7 @@ + + + + Ordinary + + + diff --git a/tests/integration/test_cluster_copier/task0_description.xml b/tests/integration/test_cluster_copier/task0_description.xml index 72eff8d464d..d56053ffd39 100644 --- a/tests/integration/test_cluster_copier/task0_description.xml +++ b/tests/integration/test_cluster_copier/task0_description.xml @@ -33,7 +33,7 @@ 3 4 5 6 1 2 0 - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 + ENGINE=ReplicatedMergeTree PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 d + 1 @@ -93,4 +93,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/tests/integration/test_cluster_copier/task_month_to_week_description.xml index ee134603310..26dfc7d3e00 100644 --- a/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -34,7 +34,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') + ReplicatedMergeTree PARTITION BY toMonday(date) ORDER BY d @@ -97,4 +97,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_test_block_size.xml b/tests/integration/test_cluster_copier/task_test_block_size.xml index ea63d580c1c..c9c99a083ea 100644 --- a/tests/integration/test_cluster_copier/task_test_block_size.xml +++ b/tests/integration/test_cluster_copier/task_test_block_size.xml @@ -28,7 +28,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/test_block_size', '{replica}') + ReplicatedMergeTree ORDER BY d PARTITION BY partition @@ -99,4 +99,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 2a9e696ca46..88dac06f158 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -81,11 +81,11 @@ class Task1: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + - "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " + + "ENGINE=ReplicatedMergeTree " + "PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16") ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)") @@ -110,10 +110,11 @@ class Task1: class Task2: - def __init__(self, cluster): + def __init__(self, cluster, unique_zk_path): self.cluster = cluster self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition" self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read() + self.unique_zk_path = unique_zk_path def start(self): instance = cluster.instances['s0_0_0'] @@ -121,11 +122,13 @@ class Task2: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, - "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") + "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/" + self.unique_zk_path + "', " + "'{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") @@ -169,7 +172,7 @@ class Task_test_block_size: ddl_check_query(instance, """ CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}') + ENGINE=ReplicatedMergeTree ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2) instance.query( @@ -332,17 +335,17 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster), []) + execute_task(Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 3d0c5d0f5b0..035faf0bb9f 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -59,7 +59,7 @@ class TaskTrivial: for node in [source, destination]: node.query("DROP DATABASE IF EXISTS default") - node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS default") source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index 119bd7c6863..9c36da229e1 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -12,7 +12,7 @@ def start_cluster(): try: cluster.start() for node in nodes: - node.query("CREATE DATABASE IF NOT EXISTS test") + node.query("CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary") node.query("CREATE DATABASE IF NOT EXISTS atest") node.query("CREATE DATABASE IF NOT EXISTS ztest") node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log") diff --git a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql index 48e1cd65c49..8df012a8588 100644 --- a/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql +++ b/tests/queries/0_stateless/00510_materizlized_view_and_deduplication_zookeeper.sql @@ -8,10 +8,10 @@ CREATE TABLE with_deduplication(x UInt32) CREATE TABLE without_deduplication(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00510/without_deduplication', 'r1') ORDER BY x SETTINGS replicated_deduplication_window = 0; -CREATE MATERIALIZED VIEW with_deduplication_mv +CREATE MATERIALIZED VIEW with_deduplication_mv UUID '00000510-1000-4000-8000-000000000001' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/with_deduplication_mv', 'r1') ORDER BY dummy AS SELECT 0 AS dummy, countState(x) AS cnt FROM with_deduplication; -CREATE MATERIALIZED VIEW without_deduplication_mv +CREATE MATERIALIZED VIEW without_deduplication_mv UUID '00000510-1000-4000-8000-000000000002' ENGINE = ReplicatedAggregatingMergeTree('/clickhouse/tables/test_00510/without_deduplication_mv', 'r1') ORDER BY dummy AS SELECT 0 AS dummy, countState(x) AS cnt FROM without_deduplication; @@ -32,12 +32,12 @@ SELECT countMerge(cnt) FROM with_deduplication_mv; SELECT countMerge(cnt) FROM without_deduplication_mv; -- Explicit insert is deduplicated -ALTER TABLE `.inner.with_deduplication_mv` DROP PARTITION ID 'all'; -ALTER TABLE `.inner.without_deduplication_mv` DROP PARTITION ID 'all'; -INSERT INTO `.inner.with_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.with_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.without_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; -INSERT INTO `.inner.without_deduplication_mv` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +ALTER TABLE `.inner_id.00000510-1000-4000-8000-000000000001` DROP PARTITION ID 'all'; +ALTER TABLE `.inner_id.00000510-1000-4000-8000-000000000002` DROP PARTITION ID 'all'; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000001` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000001` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000002` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; +INSERT INTO `.inner_id.00000510-1000-4000-8000-000000000002` SELECT 0 AS dummy, arrayReduce('countState', [toUInt32(42)]) AS cnt; SELECT ''; SELECT countMerge(cnt) FROM with_deduplication_mv; diff --git a/tests/queries/0_stateless/00604_show_create_database.reference b/tests/queries/0_stateless/00604_show_create_database.reference index a9ad6abea25..c05b088280e 100644 --- a/tests/queries/0_stateless/00604_show_create_database.reference +++ b/tests/queries/0_stateless/00604_show_create_database.reference @@ -1 +1 @@ -CREATE DATABASE test_00604\nENGINE = Ordinary +CREATE DATABASE test_00604\nENGINE = Atomic diff --git a/tests/queries/0_stateless/00609_mv_index_in_in.sql b/tests/queries/0_stateless/00609_mv_index_in_in.sql index 7064d8e36cd..28085194327 100644 --- a/tests/queries/0_stateless/00609_mv_index_in_in.sql +++ b/tests/queries/0_stateless/00609_mv_index_in_in.sql @@ -4,11 +4,11 @@ DROP TABLE IF EXISTS test_mv_00609; create table test_00609 (a Int8) engine=Memory; insert into test_00609 values (1); -create materialized view test_mv_00609 Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; +create materialized view test_mv_00609 uuid '00000609-1000-4000-8000-000000000001' Engine=MergeTree(date, (a), 8192) populate as select a, toDate('2000-01-01') date from test_00609; select * from test_mv_00609; -- OK select * from test_mv_00609 where a in (select a from test_mv_00609); -- EMPTY (bug) -select * from ".inner.test_mv_00609" where a in (select a from test_mv_00609); -- OK +select * from ".inner_id.00000609-1000-4000-8000-000000000001" where a in (select a from test_mv_00609); -- OK DROP TABLE test_00609; DROP TABLE test_mv_00609; diff --git a/tests/queries/0_stateless/00738_lock_for_inner_table.sh b/tests/queries/0_stateless/00738_lock_for_inner_table.sh index 2f7035b6759..4570c853f31 100755 --- a/tests/queries/0_stateless/00738_lock_for_inner_table.sh +++ b/tests/queries/0_stateless/00738_lock_for_inner_table.sh @@ -7,13 +7,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) echo "DROP TABLE IF EXISTS tab_00738; DROP TABLE IF EXISTS mv; CREATE TABLE tab_00738(a Int) ENGINE = Log; -CREATE MATERIALIZED VIEW mv ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n +CREATE MATERIALIZED VIEW mv UUID '00000738-1000-4000-8000-000000000001' ENGINE = Log AS SELECT a FROM tab_00738;" | ${CLICKHOUSE_CLIENT} -n ${CLICKHOUSE_CLIENT} --query_id test_00738 --query "INSERT INTO tab_00738 SELECT number FROM numbers(10000000)" & function drop() { - ${CLICKHOUSE_CLIENT} --query "DROP TABLE \`.inner.mv\`" -n + ${CLICKHOUSE_CLIENT} --query "DROP TABLE \`.inner_id.00000738-1000-4000-8000-000000000001\`" -n } function wait_for_query_to_start() diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.reference b/tests/queries/0_stateless/01190_full_attach_syntax.reference index 619861849c8..4e6eabcd6f0 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.reference +++ b/tests/queries/0_stateless/01190_full_attach_syntax.reference @@ -1,13 +1,13 @@ CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) -CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log -CREATE TABLE default.log\n(\n `s` String\n)\nENGINE = Log() +CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log +CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log() test -CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE TABLE test_01190.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date MATERIALIZED \'2000-01-01\'\n)\nENGINE = MergeTree(d, (key, s, n), 1) [1,2] Hello 2 -CREATE TABLE default.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) -CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log -CREATE MATERIALIZED VIEW default.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM default.log -CREATE MATERIALIZED VIEW default.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM default.mt -CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 -CREATE LIVE VIEW default.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE TABLE test_01190.mt\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = MergeTree(d, (key, s, n), 1) +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.log +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `s` String\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.log +CREATE MATERIALIZED VIEW test_01190.mv\n(\n `key` Array(UInt8),\n `s` String,\n `n` UInt64,\n `d` Date\n)\nENGINE = Null AS\nSELECT *\nFROM test_01190.mt +CREATE LIVE VIEW test_01190.lv\n(\n `1` UInt8\n) AS\nSELECT 1 +CREATE LIVE VIEW test_01190.lv\n(\n `1` UInt8\n) AS\nSELECT 1 diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql index 3a91eccc8cd..225e66bed5c 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.sql +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -1,5 +1,6 @@ DROP DATABASE IF EXISTS test_01190; -CREATE DATABASE test_01190; +CREATE DATABASE test_01190 ENGINE=Ordinary; +USE test_01190; CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; @@ -14,14 +15,6 @@ ATTACH DICTIONARY test_01190.dict (key UInt64 DEFAULT 0, col UInt8 DEFAULT 42) P ATTACH DICTIONARY test_01190.dict; SHOW CREATE DICTIONARY test_01190.dict; -DROP DATABASE test_01190; - - -DROP TABLE IF EXISTS log; -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS mv; -DROP TABLE IF EXISTS lv; - CREATE TABLE log ENGINE = Log AS SELECT 'test' AS s; SHOW CREATE log; DETACH TABLE log; @@ -58,9 +51,6 @@ DETACH VIEW lv; ATTACH LIVE VIEW lv AS SELECT 1; SHOW CREATE lv; -DROP TABLE log; -DROP TABLE mt; -DROP TABLE mv; -DROP TABLE lv; +DROP DATABASE test_01190; diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql index cf8b2a471c4..55689411ad6 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224_dictionary; -CREATE DATABASE dict_db_01224; +CREATE DATABASE dict_db_01224 ENGINE=Ordinary; CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index a494511ebd8..8980a9fd70b 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01225; DROP DATABASE IF EXISTS dict_db_01225_dictionary; -CREATE DATABASE dict_db_01225; +CREATE DATABASE dict_db_01225 ENGINE=Ordinary; CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary; CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference index e3f4955d4cf..fb993e8d572 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.reference @@ -1,3 +1,3 @@ -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(0., 1.) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 -CREATE TABLE default.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(0., 1.) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01249.bloom_filter_idx_good\n(\n `u64` UInt64,\n `i32` Int32,\n `f64` Float64,\n `d` Decimal(10, 2),\n `s` String,\n `e` Enum8(\'a\' = 1, \'b\' = 2, \'c\' = 3),\n `dt` Date,\n INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1\n)\nENGINE = MergeTree()\nORDER BY u64\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql index b60fbc05457..7bf334582a2 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql @@ -1,3 +1,7 @@ +DROP DATABASE IF EXISTS test_01249; +CREATE DATABASE test_01249 ENGINE=Ordinary; +USE test_01249; + CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(0, 1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 42 } CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(-0.1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 36 } CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 36 } @@ -14,4 +18,4 @@ DROP TABLE IF EXISTS bloom_filter_idx_good; ATTACH TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(1.01) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; SHOW CREATE TABLE bloom_filter_idx_good; -DROP TABLE IF EXISTS bloom_filter_idx_good; +DROP DATABASE test_01249; diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index 5bbec57a236..ce1f08b4c51 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -5,16 +5,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r;" +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" function thread1() { - while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE r (x UInt64) ENGINE = ReplicatedMergeTree('/test_01320/table', 'r') ORDER BY x; DROP TABLE r;"; done + while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test_01320/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done } function thread2() { - while true; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA r" 2>/dev/null; done + while true; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA test_01320.r" 2>/dev/null; done } export -f thread1 @@ -25,4 +26,4 @@ timeout 10 bash -c thread2 & wait -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS r;" +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index e4713b2d960..4ffa2bd74cb 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -90,17 +90,17 @@ ], "database-atomic": [ /// Inner tables of materialized views have different names - "00738_lock_for_inner_table", - "00609_mv_index_in_in", - "00510_materizlized_view_and_deduplication_zookeeper", + ///"00738_lock_for_inner_table", + ///"00609_mv_index_in_in", + ///"00510_materizlized_view_and_deduplication_zookeeper", /// Different database engine - "00604_show_create_database", + ///"00604_show_create_database", /// UUID must be specified in ATTACH TABLE - "01190_full_attach_syntax", + ///"01190_full_attach_syntax", /// Assumes blocking DROP - "01320_create_sync_race_condition", + ///"01320_create_sync_race_condition", /// Internal distionary name is different - "01225_show_create_table_from_dictionary", - "01224_no_superfluous_dict_reload" + ///"01225_show_create_table_from_dictionary", + ///"01224_no_superfluous_dict_reload" ] } From d87f3660ee2d75c993bc2b6356dae37a9186e6e3 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 21 Sep 2020 23:19:46 +0300 Subject: [PATCH 019/131] done --- .../test.py | 346 ------------------ .../__init__.py | 0 .../common.py | 239 ++++++++++++ .../configs/config.xml | 0 .../configs/dictionaries/.gitkeep | 0 .../configs/disable_ssl_verification.xml | 0 .../configs/enable_dictionaries.xml | 0 .../configs/users.xml | 0 .../test_cassandra.py | 82 +++++ .../test_clickhouse_local.py | 82 +++++ .../test_clickhouse_remote.py | 84 +++++ .../test_executable_cache.py | 82 +++++ .../test_executable_hashed.py | 82 +++++ .../test_file.py | 82 +++++ .../test_http.py | 84 +++++ .../test_https.py | 84 +++++ .../test_mongo.py | 82 +++++ .../test_mongo_uri.py | 82 +++++ .../test_mysql.py | 82 +++++ 19 files changed, 1147 insertions(+), 346 deletions(-) delete mode 100644 tests/integration/test_dictionaries_all_layouts_and_sources/test.py rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/__init__.py (100%) create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/common.py rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/configs/config.xml (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/configs/dictionaries/.gitkeep (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/configs/disable_ssl_verification.xml (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/configs/enable_dictionaries.xml (100%) rename tests/integration/{test_dictionaries_all_layouts_and_sources => test_dictionaries_all_layouts_separate_sources}/configs/users.xml (100%) create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py create mode 100644 tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py b/tests/integration/test_dictionaries_all_layouts_and_sources/test.py deleted file mode 100644 index 5880ead7c5a..00000000000 --- a/tests/integration/test_dictionaries_all_layouts_and_sources/test.py +++ /dev/null @@ -1,346 +0,0 @@ -import math -import os - -import pytest -from helpers.cluster import ClickHouseCluster -from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout -from helpers.external_sources import SourceMongo, SourceMongoURI, SourceHTTP, SourceHTTPS, SourceCassandra -from helpers.external_sources import SourceMySQL, SourceClickHouse, SourceFile, SourceExecutableCache, \ - SourceExecutableHashed - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -dict_configs_path = os.path.join(SCRIPT_DIR, 'configs/dictionaries') - -FIELDS = { - "simple": [ - Field("KeyField", 'UInt64', is_key=True, default_value_for_get=9999999), - Field("UInt8_", 'UInt8', default_value_for_get=55), - Field("UInt16_", 'UInt16', default_value_for_get=66), - Field("UInt32_", 'UInt32', default_value_for_get=77), - Field("UInt64_", 'UInt64', default_value_for_get=88), - Field("Int8_", 'Int8', default_value_for_get=-55), - Field("Int16_", 'Int16', default_value_for_get=-66), - Field("Int32_", 'Int32', default_value_for_get=-77), - Field("Int64_", 'Int64', default_value_for_get=-88), - Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'), - Field("Date_", 'Date', default_value_for_get='2018-12-30'), - Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'), - Field("String_", 'String', default_value_for_get='hi'), - Field("Float32_", 'Float32', default_value_for_get=555.11), - Field("Float64_", 'Float64', default_value_for_get=777.11), - Field("ParentKeyField", "UInt64", default_value_for_get=444, hierarchical=True) - ], - "complex": [ - Field("KeyField1", 'UInt64', is_key=True, default_value_for_get=9999999), - Field("KeyField2", 'String', is_key=True, default_value_for_get='xxxxxxxxx'), - Field("UInt8_", 'UInt8', default_value_for_get=55), - Field("UInt16_", 'UInt16', default_value_for_get=66), - Field("UInt32_", 'UInt32', default_value_for_get=77), - Field("UInt64_", 'UInt64', default_value_for_get=88), - Field("Int8_", 'Int8', default_value_for_get=-55), - Field("Int16_", 'Int16', default_value_for_get=-66), - Field("Int32_", 'Int32', default_value_for_get=-77), - Field("Int64_", 'Int64', default_value_for_get=-88), - Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'), - Field("Date_", 'Date', default_value_for_get='2018-12-30'), - Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'), - Field("String_", 'String', default_value_for_get='hi'), - Field("Float32_", 'Float32', default_value_for_get=555.11), - Field("Float64_", 'Float64', default_value_for_get=777.11), - ], - "ranged": [ - Field("KeyField1", 'UInt64', is_key=True), - Field("KeyField2", 'Date', is_range_key=True), - Field("StartDate", 'Date', range_hash_type='min'), - Field("EndDate", 'Date', range_hash_type='max'), - Field("UInt8_", 'UInt8', default_value_for_get=55), - Field("UInt16_", 'UInt16', default_value_for_get=66), - Field("UInt32_", 'UInt32', default_value_for_get=77), - Field("UInt64_", 'UInt64', default_value_for_get=88), - Field("Int8_", 'Int8', default_value_for_get=-55), - Field("Int16_", 'Int16', default_value_for_get=-66), - Field("Int32_", 'Int32', default_value_for_get=-77), - Field("Int64_", 'Int64', default_value_for_get=-88), - Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'), - Field("Date_", 'Date', default_value_for_get='2018-12-30'), - Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'), - Field("String_", 'String', default_value_for_get='hi'), - Field("Float32_", 'Float32', default_value_for_get=555.11), - Field("Float64_", 'Float64', default_value_for_get=777.11), - ] -} - -VALUES = { - "simple": [ - [1, 22, 333, 4444, 55555, -6, -77, - -888, -999, '550e8400-e29b-41d4-a716-446655440003', - '1973-06-28', '1985-02-28 23:43:25', 'hello', 22.543, 3332154213.4, 0], - [2, 3, 4, 5, 6, -7, -8, - -9, -10, '550e8400-e29b-41d4-a716-446655440002', - '1978-06-28', '1986-02-28 23:42:25', 'hello', 21.543, 3222154213.4, 1] - ], - "complex": [ - [1, 'world', 22, 333, 4444, 55555, -6, - -77, -888, -999, '550e8400-e29b-41d4-a716-446655440003', - '1973-06-28', '1985-02-28 23:43:25', - 'hello', 22.543, 3332154213.4], - [2, 'qwerty2', 52, 2345, 6544, 9191991, -2, - -717, -81818, -92929, '550e8400-e29b-41d4-a716-446655440007', - '1975-09-28', '2000-02-28 23:33:24', - 'my', 255.543, 3332221.44] - - ], - "ranged": [ - [1, '2019-02-10', '2019-02-01', '2019-02-28', - 22, 333, 4444, 55555, -6, -77, -888, -999, - '550e8400-e29b-41d4-a716-446655440003', - '1973-06-28', '1985-02-28 23:43:25', 'hello', - 22.543, 3332154213.4], - [2, '2019-04-10', '2019-04-01', '2019-04-28', - 11, 3223, 41444, 52515, -65, -747, -8388, -9099, - '550e8400-e29b-41d4-a716-446655440004', - '1973-06-29', '2002-02-28 23:23:25', '!!!!', - 32.543, 3332543.4] - ] -} - -LAYOUTS = [ - Layout("flat"), - Layout("hashed"), - Layout("cache"), - Layout("complex_key_hashed"), - Layout("complex_key_cache"), - Layout("range_hashed"), - Layout("direct"), - Layout("complex_key_direct") -] - -SOURCES = [ - SourceCassandra("Cassandra", "localhost", "9043", "cassandra1", "9042", "", ""), - SourceMongo("MongoDB", "localhost", "27018", "mongo1", "27017", "root", "clickhouse"), - SourceMongoURI("MongoDB_URI", "localhost", "27018", "mongo1", "27017", "root", "clickhouse"), - SourceMySQL("MySQL", "localhost", "3308", "mysql1", "3306", "root", "clickhouse"), - SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse1", "9000", "default", ""), - SourceClickHouse("LocalClickHouse", "localhost", "9000", "node", "9000", "default", ""), - SourceFile("File", "localhost", "9000", "node", "9000", "", ""), - SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "node", "9000", "", ""), - SourceExecutableCache("ExecutableCache", "localhost", "9000", "node", "9000", "", ""), - SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse1", "9000", "", ""), - SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse1", "9000", "", ""), -] - -DICTIONARIES = [] - -cluster = None -node = None - - -def get_dict(source, layout, fields, suffix_name=''): - global dict_configs_path - - structure = DictionaryStructure(layout, fields) - dict_name = source.name + "_" + layout.name + '_' + suffix_name - dict_path = os.path.join(dict_configs_path, dict_name + '.xml') - dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) - dictionary.generate_config() - return dictionary - - -def setup_module(module): - global DICTIONARIES - global cluster - global node - global dict_configs_path - - for f in os.listdir(dict_configs_path): - os.remove(os.path.join(dict_configs_path, f)) - - for layout in LAYOUTS: - for source in SOURCES: - if source.compatible_with_layout(layout): - DICTIONARIES.append(get_dict(source, layout, FIELDS[layout.layout_type])) - else: - print "Source", source.name, "incompatible with layout", layout.name - - cluster = ClickHouseCluster(__file__) - - main_configs = [] - main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - cluster.add_instance('clickhouse1', main_configs=main_configs) - - dictionaries = [] - for fname in os.listdir(dict_configs_path): - dictionaries.append(os.path.join(dict_configs_path, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True, - with_mongo=True, with_redis=True, with_cassandra=True) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - for dictionary in DICTIONARIES: - print "Preparing", dictionary.name - dictionary.prepare_source(cluster) - print "Prepared" - - yield cluster - - finally: - cluster.shutdown() - - -def get_dictionaries(fold, total_folds, all_dicts): - chunk_len = int(math.ceil(len(all_dicts) / float(total_folds))) - if chunk_len * fold >= len(all_dicts): - return [] - return all_dicts[fold * chunk_len: (fold + 1) * chunk_len] - - -def remove_mysql_dicts(): - """ - We have false-positive race condition in our openSSL version. - MySQL dictionary use OpenSSL, so to prevent known failure we - disable tests for these dictionaries. - - Read of size 8 at 0x7b3c00005dd0 by thread T61 (mutexes: write M1010349240585225536): - #0 EVP_CIPHER_mode (clickhouse+0x13b2223b) - #1 do_ssl3_write (clickhouse+0x13a137bc) - #2 ssl3_write_bytes (clickhouse+0x13a12387) - #3 ssl3_write (clickhouse+0x139db0e6) - #4 ssl_write_internal (clickhouse+0x139eddce) - #5 SSL_write (clickhouse+0x139edf20) - #6 ma_tls_write (clickhouse+0x139c7557) - #7 ma_pvio_tls_write (clickhouse+0x139a8f59) - #8 ma_pvio_write (clickhouse+0x139a8488) - #9 ma_net_real_write (clickhouse+0x139a4e2c) - #10 ma_net_write_command (clickhouse+0x139a546d) - #11 mthd_my_send_cmd (clickhouse+0x13992546) - #12 mysql_close_slow_part (clickhouse+0x13999afd) - #13 mysql_close (clickhouse+0x13999071) - #14 mysqlxx::Connection::~Connection() (clickhouse+0x1370f814) - #15 mysqlxx::Pool::~Pool() (clickhouse+0x13715a7b) - - TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed - """ - - # global DICTIONARIES - # DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")] - - -@pytest.mark.parametrize("fold", list(range(10))) -def test_simple_dictionaries(started_cluster, fold): - if node.is_built_with_thread_sanitizer(): - remove_mysql_dicts() - - fields = FIELDS["simple"] - values = VALUES["simple"] - data = [Row(fields, vals) for vals in values] - - all_simple_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "simple"] - simple_dicts = get_dictionaries(fold, 10, all_simple_dicts) - - print "Length of dicts:", len(simple_dicts) - for dct in simple_dicts: - dct.load_data(data) - - node.query("system reload dictionaries") - - queries_with_answers = [] - for dct in simple_dicts: - for row in data: - for field in fields: - if not field.is_key: - for query in dct.get_select_get_queries(field, row): - queries_with_answers.append((query, row.get_value_by_name(field.name))) - - for query in dct.get_select_has_queries(field, row): - queries_with_answers.append((query, 1)) - - for query in dct.get_select_get_or_default_queries(field, row): - queries_with_answers.append((query, field.default_value_for_get)) - for query in dct.get_hierarchical_queries(data[0]): - queries_with_answers.append((query, [1])) - - for query in dct.get_hierarchical_queries(data[1]): - queries_with_answers.append((query, [2, 1])) - - for query in dct.get_is_in_queries(data[0], data[1]): - queries_with_answers.append((query, 0)) - - for query in dct.get_is_in_queries(data[1], data[0]): - queries_with_answers.append((query, 1)) - - for query, answer in queries_with_answers: - print query - if isinstance(answer, list): - answer = str(answer).replace(' ', '') - assert node.query(query) == str(answer) + '\n' - - -@pytest.mark.parametrize("fold", list(range(10))) -def test_complex_dictionaries(started_cluster, fold): - if node.is_built_with_thread_sanitizer(): - remove_mysql_dicts() - - fields = FIELDS["complex"] - values = VALUES["complex"] - data = [Row(fields, vals) for vals in values] - - all_complex_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex"] - complex_dicts = get_dictionaries(fold, 10, all_complex_dicts) - - for dct in complex_dicts: - dct.load_data(data) - - node.query("system reload dictionaries") - - queries_with_answers = [] - for dct in complex_dicts: - for row in data: - for field in fields: - if not field.is_key: - for query in dct.get_select_get_queries(field, row): - queries_with_answers.append((query, row.get_value_by_name(field.name))) - - for query in dct.get_select_has_queries(field, row): - queries_with_answers.append((query, 1)) - - for query in dct.get_select_get_or_default_queries(field, row): - queries_with_answers.append((query, field.default_value_for_get)) - - for query, answer in queries_with_answers: - print query - assert node.query(query) == str(answer) + '\n' - - -@pytest.mark.parametrize("fold", list(range(10))) -def test_ranged_dictionaries(started_cluster, fold): - if node.is_built_with_thread_sanitizer(): - remove_mysql_dicts() - - fields = FIELDS["ranged"] - values = VALUES["ranged"] - data = [Row(fields, vals) for vals in values] - - all_ranged_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "ranged"] - ranged_dicts = get_dictionaries(fold, 10, all_ranged_dicts) - - for dct in ranged_dicts: - dct.load_data(data) - - node.query("system reload dictionaries") - - queries_with_answers = [] - for dct in ranged_dicts: - for row in data: - for field in fields: - if not field.is_key and not field.is_range: - for query in dct.get_select_get_queries(field, row): - queries_with_answers.append((query, row.get_value_by_name(field.name))) - - for query, answer in queries_with_answers: - print query - assert node.query(query) == str(answer) + '\n' diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/__init__.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/__init__.py similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/__init__.py rename to tests/integration/test_dictionaries_all_layouts_separate_sources/__init__.py diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py new file mode 100644 index 00000000000..0411b5d9475 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -0,0 +1,239 @@ +import os + +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout + +KEY_FIELDS = { + "simple": [ + Field("KeyField", 'UInt64', is_key=True, default_value_for_get=9999999) + ], + "complex": [ + Field("KeyField1", 'UInt64', is_key=True, default_value_for_get=9999999), + Field("KeyField2", 'String', is_key=True, default_value_for_get='xxxxxxxxx') + ], + "ranged": [ + Field("KeyField1", 'UInt64', is_key=True), + Field("KeyField2", 'Date', is_range_key=True) + ] +} + +START_FIELDS = { + "simple": [], + "complex": [], + "ranged" : [ + Field("StartDate", 'Date', range_hash_type='min'), + Field("EndDate", 'Date', range_hash_type='max') + ] +} + +MIDDLE_FIELDS = [ + Field("UInt8_", 'UInt8', default_value_for_get=55), + Field("UInt16_", 'UInt16', default_value_for_get=66), + Field("UInt32_", 'UInt32', default_value_for_get=77), + Field("UInt64_", 'UInt64', default_value_for_get=88), + Field("Int8_", 'Int8', default_value_for_get=-55), + Field("Int16_", 'Int16', default_value_for_get=-66), + Field("Int32_", 'Int32', default_value_for_get=-77), + Field("Int64_", 'Int64', default_value_for_get=-88), + Field("UUID_", 'UUID', default_value_for_get='550e8400-0000-0000-0000-000000000000'), + Field("Date_", 'Date', default_value_for_get='2018-12-30'), + Field("DateTime_", 'DateTime', default_value_for_get='2018-12-30 00:00:00'), + Field("String_", 'String', default_value_for_get='hi'), + Field("Float32_", 'Float32', default_value_for_get=555.11), + Field("Float64_", 'Float64', default_value_for_get=777.11), +] + +END_FIELDS = { + "simple" : [ + Field("ParentKeyField", "UInt64", default_value_for_get=444, hierarchical=True) + ], + "complex" : [], + "ranged" : [] +} + +LAYOUTS_SIMPLE = ["flat", "hashed", "cache", "direct"] +LAYOUTS_COMPLEX = ["complex_key_hashed", "complex_key_cache", "complex_key_direct"] +LAYOUTS_RANGED = ["range_hashed"] + +VALUES = { + "simple": [ + [1, 22, 333, 4444, 55555, -6, -77, + -888, -999, '550e8400-e29b-41d4-a716-446655440003', + '1973-06-28', '1985-02-28 23:43:25', 'hello', 22.543, 3332154213.4, 0], + [2, 3, 4, 5, 6, -7, -8, + -9, -10, '550e8400-e29b-41d4-a716-446655440002', + '1978-06-28', '1986-02-28 23:42:25', 'hello', 21.543, 3222154213.4, 1] + ], + "complex": [ + [1, 'world', 22, 333, 4444, 55555, -6, + -77, -888, -999, '550e8400-e29b-41d4-a716-446655440003', + '1973-06-28', '1985-02-28 23:43:25', + 'hello', 22.543, 3332154213.4], + [2, 'qwerty2', 52, 2345, 6544, 9191991, -2, + -717, -81818, -92929, '550e8400-e29b-41d4-a716-446655440007', + '1975-09-28', '2000-02-28 23:33:24', + 'my', 255.543, 3332221.44] + ], + "ranged": [ + [1, '2019-02-10', '2019-02-01', '2019-02-28', + 22, 333, 4444, 55555, -6, -77, -888, -999, + '550e8400-e29b-41d4-a716-446655440003', + '1973-06-28', '1985-02-28 23:43:25', 'hello', + 22.543, 3332154213.4], + [2, '2019-04-10', '2019-04-01', '2019-04-28', + 11, 3223, 41444, 52515, -65, -747, -8388, -9099, + '550e8400-e29b-41d4-a716-446655440004', + '1973-06-29', '2002-02-28 23:23:25', '!!!!', + 32.543, 3332543.4] + ] +} + + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/dictionaries') + +def get_dict(source, layout, fields, suffix_name=''): + global DICT_CONFIG_PATH + structure = DictionaryStructure(layout, fields) + dict_name = source.name + "_" + layout.name + '_' + suffix_name + dict_path = os.path.join(DICT_CONFIG_PATH, dict_name + '.xml') + dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) + dictionary.generate_config() + return dictionary + +class SimpleLayoutTester: + def __init__(self): + self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] + self.values = VALUES["simple"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + + def create_dictionaries(self, source_): + for layout in LAYOUTS_SIMPLE: + if source_.compatible_with_layout(Layout(layout)): + self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + + def prepare(self, cluster_): + for _, dictionary in self.layout_to_dictionary.items(): + dictionary.prepare_source(cluster_) + dictionary.load_data(self.data) + + def execute(self, layout_name, node): + if not self.layout_to_dictionary.has_key(layout_name): + raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) + + dct = self.layout_to_dictionary[layout_name] + + node.query("system reload dictionaries") + queries_with_answers = [] + + for row in self.data: + for field in self.fields: + if not field.is_key: + for query in dct.get_select_get_queries(field, row): + queries_with_answers.append((query, row.get_value_by_name(field.name))) + + for query in dct.get_select_has_queries(field, row): + queries_with_answers.append((query, 1)) + + for query in dct.get_select_get_or_default_queries(field, row): + queries_with_answers.append((query, field.default_value_for_get)) + + for query in dct.get_hierarchical_queries(self.data[0]): + queries_with_answers.append((query, [1])) + + for query in dct.get_hierarchical_queries(self.data[1]): + queries_with_answers.append((query, [2, 1])) + + for query in dct.get_is_in_queries(self.data[0], self.data[1]): + queries_with_answers.append((query, 0)) + + for query in dct.get_is_in_queries(self.data[1], self.data[0]): + queries_with_answers.append((query, 1)) + + for query, answer in queries_with_answers: + # print query + if isinstance(answer, list): + answer = str(answer).replace(' ', '') + assert node.query(query) == str(answer) + '\n' + + +class ComplexLayoutTester: + def __init__(self): + self.fields = KEY_FIELDS["complex"] + START_FIELDS["complex"] + MIDDLE_FIELDS + END_FIELDS["complex"] + self.values = VALUES["complex"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + + def create_dictionaries(self, source_): + for layout in LAYOUTS_COMPLEX: + if source_.compatible_with_layout(Layout(layout)): + self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + + def prepare(self, cluster_): + for _, dictionary in self.layout_to_dictionary.items(): + dictionary.prepare_source(cluster_) + dictionary.load_data(self.data) + + def execute(self, layout_name, node): + if not self.layout_to_dictionary.has_key(layout_name): + raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) + + dct = self.layout_to_dictionary[layout_name] + + node.query("system reload dictionaries") + queries_with_answers = [] + + for row in self.data: + for field in self.fields: + if not field.is_key: + for query in dct.get_select_get_queries(field, row): + queries_with_answers.append((query, row.get_value_by_name(field.name))) + + for query in dct.get_select_has_queries(field, row): + queries_with_answers.append((query, 1)) + + for query in dct.get_select_get_or_default_queries(field, row): + queries_with_answers.append((query, field.default_value_for_get)) + + for query, answer in queries_with_answers: + # print query + assert node.query(query) == str(answer) + '\n' + + +class RangedLayoutTester: + def __init__(self): + self.fields = KEY_FIELDS["ranged"] + START_FIELDS["ranged"] + MIDDLE_FIELDS + END_FIELDS["ranged"] + self.values = VALUES["ranged"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + + def create_dictionaries(self, source_): + for layout in LAYOUTS_RANGED: + if source_.compatible_with_layout(Layout(layout)): + self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + + def prepare(self, cluster_): + for _, dictionary in self.layout_to_dictionary.items(): + dictionary.prepare_source(cluster_) + dictionary.load_data(self.data) + + def execute(self, layout_name, node): + + if not self.layout_to_dictionary.has_key(layout_name): + raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) + + dct = self.layout_to_dictionary[layout_name] + + node.query("system reload dictionaries") + + queries_with_answers = [] + for row in self.data: + for field in self.fields: + if not field.is_key and not field.is_range: + for query in dct.get_select_get_queries(field, row): + queries_with_answers.append((query, row.get_value_by_name(field.name))) + + for query, answer in queries_with_answers: + # print query + assert node.query(query) == str(answer) + '\n' + diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/config.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/configs/config.xml rename to tests/integration/test_dictionaries_all_layouts_separate_sources/configs/config.xml diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/dictionaries/.gitkeep b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/configs/dictionaries/.gitkeep rename to tests/integration/test_dictionaries_all_layouts_separate_sources/configs/dictionaries/.gitkeep diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/configs/disable_ssl_verification.xml rename to tests/integration/test_dictionaries_all_layouts_separate_sources/configs/disable_ssl_verification.xml diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/configs/enable_dictionaries.xml rename to tests/integration/test_dictionaries_all_layouts_separate_sources/configs/enable_dictionaries.xml diff --git a/tests/integration/test_dictionaries_all_layouts_and_sources/configs/users.xml b/tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml similarity index 100% rename from tests/integration/test_dictionaries_all_layouts_and_sources/configs/users.xml rename to tests/integration/test_dictionaries_all_layouts_separate_sources/configs/users.xml diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py new file mode 100644 index 00000000000..c6b2ed370f4 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceCassandra + +SOURCE = SourceCassandra("Cassandra", "localhost", "9043", "cassandra1", "9042", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py new file mode 100644 index 00000000000..c8597708c04 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceClickHouse + +SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "node", "9000", "default", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py new file mode 100644 index 00000000000..4f190e4c332 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -0,0 +1,84 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceClickHouse + +SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse1", "9000", "default", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + cluster.add_instance('clickhouse1', main_configs=main_configs) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_SIMPLE).difference(set("cache"))) ) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_COMPLEX).difference(set("complex_key_cache")))) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py new file mode 100644 index 00000000000..87b73d3acc1 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceExecutableCache + +SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "node", "9000", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py new file mode 100644 index 00000000000..5ab5ee545e7 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceExecutableHashed + +SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "node", "9000", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py new file mode 100644 index 00000000000..893d095a143 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceFile + +SOURCE = SourceFile("File", "localhost", "9000", "node", "9000", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_SIMPLE).difference(set("cache, direct"))) ) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_SIMPLE).difference(set("complex_key_cache, complex_key_direct")))) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py new file mode 100644 index 00000000000..8c7519fd0a7 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -0,0 +1,84 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceHTTP + +SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse1", "9000", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + cluster.add_instance('clickhouse1', main_configs=main_configs) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py new file mode 100644 index 00000000000..c5280aa0076 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -0,0 +1,84 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceHTTPS + +SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse1", "9000", "", "") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + cluster.add_instance('clickhouse1', main_configs=main_configs) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py new file mode 100644 index 00000000000..ffa376dcdb3 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceMongo + +SOURCE = SourceMongo("MongoDB", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py new file mode 100644 index 00000000000..53edfc4bbd2 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceMongoURI + +SOURCE = SourceMongoURI("MongoDB_URI", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py new file mode 100644 index 00000000000..69e2543f226 --- /dev/null +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -0,0 +1,82 @@ +import os +import math +import pytest + +from .common import * + +from helpers.cluster import ClickHouseCluster +from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout +from helpers.external_sources import SourceMySQL + +SOURCE = SourceMySQL("MySQL", "localhost", "3308", "mysql1", "3306", "root", "clickhouse") + +cluster = None +node = None +simple_tester = None +complex_tester = None +ranged_tester = None + + +def setup_module(module): + global cluster + global node + global simple_tester + global complex_tester + global ranged_tester + + for f in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, f)) + + simple_tester = SimpleLayoutTester() + simple_tester.create_dictionaries(SOURCE) + + complex_tester = ComplexLayoutTester() + complex_tester.create_dictionaries(SOURCE) + + ranged_tester = RangedLayoutTester() + ranged_tester.create_dictionaries(SOURCE) + # Since that all .xml configs were created + + cluster = ClickHouseCluster(__file__) + + dictionaries = [] + main_configs = [] + main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) + + for fname in os.listdir(DICT_CONFIG_PATH): + dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) + + +def teardown_module(module): + global DICT_CONFIG_PATH + for fname in os.listdir(DICT_CONFIG_PATH): + os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + simple_tester.prepare(cluster) + complex_tester.prepare(cluster) + ranged_tester.prepare(cluster) + + yield cluster + + finally: + cluster.shutdown() + +@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +def test_simple(started_cluster, layout_name): + simple_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +def test_complex(started_cluster, layout_name): + complex_tester.execute(layout_name, node) + +@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) +def test_ranged(started_cluster, layout_name): + ranged_tester.execute(layout_name, node) From 91cbd49e85ee3de6c93719e17bbf7e7e422d889e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 22 Sep 2020 00:09:50 +0300 Subject: [PATCH 020/131] fix tests --- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- .../System/StorageSystemZooKeeper.cpp | 20 +-- .../test.py | 23 ++-- .../integration/test_distributed_ddl/test.py | 12 +- .../configs/users.d/not_optimize_count.xml | 1 - .../test_polymorphic_parts/test.py | 45 +++--- .../configs/config.d/remote_servers.xml | 20 +++ tests/integration/test_quorum_inserts/test.py | 66 +++++---- .../configs/config.d/storage_conf.xml | 24 ++++ .../test_replicated_merge_tree_s3/test.py | 13 +- tests/integration/test_row_policy/test.py | 4 +- .../01038_dictionary_lifetime_min_zero_sec.sh | 28 ++-- .../01268_dictionary_direct_layout.sql | 128 +++++++++--------- tests/queries/skip_list.json | 13 -- 14 files changed, 216 insertions(+), 185 deletions(-) create mode 100644 tests/integration/test_quorum_inserts/configs/config.d/remote_servers.xml diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d7230940bb2..cc39d6fc8a3 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -858,11 +858,9 @@ BlockIO InterpreterCreateQuery::execute() auto & create = query_ptr->as(); if (!create.cluster.empty()) { - /// Allows to execute ON CLUSTER queries during version upgrade - bool force_backward_compatibility = !context.getSettingsRef().show_table_uuid_in_table_create_query_if_not_nil; /// For CREATE query generate UUID on initiator, so it will be the same on all hosts. /// It will be ignored if database does not support UUIDs. - if (!force_backward_compatibility && !create.attach && create.uuid == UUIDHelpers::Nil) + if (!create.attach && create.uuid == UUIDHelpers::Nil) create.uuid = UUIDHelpers::generateV4(); return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess()); } diff --git a/src/Storages/System/StorageSystemZooKeeper.cpp b/src/Storages/System/StorageSystemZooKeeper.cpp index 17ab4ed4efb..81a42f1fe63 100644 --- a/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/src/Storages/System/StorageSystemZooKeeper.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include @@ -42,7 +43,7 @@ NamesAndTypesList StorageSystemZooKeeper::getNamesAndTypes() } -static bool extractPathImpl(const IAST & elem, String & res) +static bool extractPathImpl(const IAST & elem, String & res, const Context & context) { const auto * function = elem.as(); if (!function) @@ -51,7 +52,7 @@ static bool extractPathImpl(const IAST & elem, String & res) if (function->name == "and") { for (const auto & child : function->arguments->children) - if (extractPathImpl(*child, res)) + if (extractPathImpl(*child, res, context)) return true; return false; @@ -60,23 +61,24 @@ static bool extractPathImpl(const IAST & elem, String & res) if (function->name == "equals") { const auto & args = function->arguments->as(); - const IAST * value; + ASTPtr value; if (args.children.size() != 2) return false; const ASTIdentifier * ident; if ((ident = args.children.at(0)->as())) - value = args.children.at(1).get(); + value = args.children.at(1); else if ((ident = args.children.at(1)->as())) - value = args.children.at(0).get(); + value = args.children.at(0); else return false; if (ident->name != "path") return false; - const auto * literal = value->as(); + auto evaluated = evaluateConstantExpressionAsLiteral(value, context); + const auto * literal = evaluated->as(); if (!literal) return false; @@ -93,20 +95,20 @@ static bool extractPathImpl(const IAST & elem, String & res) /** Retrieve from the query a condition of the form `path = 'path'`, from conjunctions in the WHERE clause. */ -static String extractPath(const ASTPtr & query) +static String extractPath(const ASTPtr & query, const Context & context) { const auto & select = query->as(); if (!select.where()) return ""; String res; - return extractPathImpl(*select.where(), res) ? res : ""; + return extractPathImpl(*select.where(), res, context) ? res : ""; } void StorageSystemZooKeeper::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo & query_info) const { - String path = extractPath(query_info.query); + String path = extractPath(query_info.query, context); if (path.empty()) throw Exception("SELECT from system.zookeeper table must contain condition like path = 'path' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index df8bd6ab56f..45404c1170c 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -17,6 +17,7 @@ node4 = cluster.add_instance('node4') def started_cluster(): try: cluster.start() + yield cluster finally: cluster.shutdown() @@ -141,22 +142,24 @@ def test_backup_from_old_version_config(started_cluster): def test_backup_and_alter(started_cluster): - node4.query("CREATE TABLE backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()") + node4.query("CREATE DATABASE test ENGINE=Ordinary") - node4.query("INSERT INTO backup_table VALUES(2, '2', toDate('2019-10-01'))") + node4.query("CREATE TABLE test.backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()") - node4.query("ALTER TABLE backup_table FREEZE PARTITION tuple();") + node4.query("INSERT INTO test.backup_table VALUES(2, '2', toDate('2019-10-01'))") - node4.query("ALTER TABLE backup_table DROP COLUMN C") + node4.query("ALTER TABLE test.backup_table FREEZE PARTITION tuple();") - node4.query("ALTER TABLE backup_table MODIFY COLUMN B UInt64") + node4.query("ALTER TABLE test.backup_table DROP COLUMN C") - node4.query("ALTER TABLE backup_table DROP PARTITION tuple()") + node4.query("ALTER TABLE test.backup_table MODIFY COLUMN B UInt64") + + node4.query("ALTER TABLE test.backup_table DROP PARTITION tuple()") node4.exec_in_container(['bash', '-c', - 'cp -r /var/lib/clickhouse/shadow/1/data/default/backup_table/all_1_1_0/ /var/lib/clickhouse/data/default/backup_table/detached']) + 'cp -r /var/lib/clickhouse/shadow/1/data/test/backup_table/all_1_1_0/ /var/lib/clickhouse/data/test/backup_table/detached']) - node4.query("ALTER TABLE backup_table ATTACH PARTITION tuple()") + node4.query("ALTER TABLE test.backup_table ATTACH PARTITION tuple()") - assert node4.query("SELECT sum(A) FROM backup_table") == "2\n" - assert node4.query("SELECT B + 2 FROM backup_table") == "4\n" + assert node4.query("SELECT sum(A) FROM test.backup_table") == "2\n" + assert node4.query("SELECT B + 2 FROM test.backup_table") == "4\n" diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 08027fa13ca..7f254d03015 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -326,19 +326,15 @@ def test_socket_timeout(test_cluster): def test_replicated_without_arguments(test_cluster): rules = test_cluster.pm_random_drops.pop_rules() instance = test_cluster.instances['ch1'] - test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic", - settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", - settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", - settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n", - settings={'show_table_uuid_in_table_create_query_if_not_nil': 1}) + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.pm_random_drops.push_rules(rules) diff --git a/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml b/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml index 82689093adf..5a06453b214 100644 --- a/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml +++ b/tests/integration/test_polymorphic_parts/configs/users.d/not_optimize_count.xml @@ -2,7 +2,6 @@ 0 - Ordinary diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index dbbf5c0b4ff..39745e4f3f0 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -336,7 +336,7 @@ def test_polymorphic_parts_non_adaptive(start_cluster): "Wide\t2\n") assert node1.contains_in_log( - " default.non_adaptive_table: Table can't create parts with adaptive granularity") + " default.non_adaptive_table ([0-9a-f-]*): Table can't create parts with adaptive granularity") def test_in_memory(start_cluster): @@ -408,24 +408,29 @@ def test_in_memory_wal(start_cluster): pm.partition_instances(node11, node12) check(node11, 300, 6) - wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal.bin") + wal_file = "/var/lib/clickhouse/data/default/wal_table/wal.bin" # Corrupt wal file - open(wal_file, 'rw+').truncate(os.path.getsize(wal_file) - 10) + # Truncate it to it's size minus 10 bytes + node11.exec_in_container(['bash', '-c', 'truncate --size="$(($(stat -c "%s" {}) - 10))" {}'.format(wal_file, wal_file)], + privileged=True, user='root') node11.restart_clickhouse(kill=True) # Broken part is lost, but other restored successfully check(node11, 250, 5) # WAL with blocks from 0 to 4 - broken_wal_file = os.path.join(node11.path, "database/data/default/wal_table/wal_0_4.bin") - assert os.path.exists(broken_wal_file) + broken_wal_file = "/var/lib/clickhouse/data/default/wal_table/wal_0_4.bin" + # Check file exists + node11.exec_in_container(['bash', '-c', 'test -f {}'.format(broken_wal_file)]) # Fetch lost part from replica node11.query("SYSTEM SYNC REPLICA wal_table", timeout=20) check(node11, 300, 6) # Check that new data is written to new wal, but old is still exists for restoring - assert os.path.getsize(wal_file) > 0 - assert os.path.exists(broken_wal_file) + # Check file not empty + node11.exec_in_container(['bash', '-c', 'test -s {}'.format(wal_file)]) + # Check file exists + node11.exec_in_container(['bash', '-c', 'test -f {}'.format(broken_wal_file)]) # Data is lost without WAL node11.query("ALTER TABLE wal_table MODIFY SETTING in_memory_parts_enable_wal = 0") @@ -446,8 +451,8 @@ def test_in_memory_wal_rotate(start_cluster): insert_random_data('restore_table', node11, 50) for i in range(5): - wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) - assert os.path.exists(wal_file) + # Check file exists + node11.exec_in_container(['bash', '-c', 'test -f /var/lib/clickhouse/data/default/restore_table/wal_{0}_{0}.bin'.format(i)]) for node in [node11, node12]: node.query( @@ -459,13 +464,14 @@ def test_in_memory_wal_rotate(start_cluster): node11.restart_clickhouse(kill=True) for i in range(5): - wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal_{0}_{0}.bin".format(i)) - assert not os.path.exists(wal_file) + # check file doesn't exist + node11.exec_in_container(['bash', '-c', 'test ! -e /var/lib/clickhouse/data/default/restore_table/wal_{0}_{0}.bin'.format(i)]) # New wal file was created and ready to write part to it - wal_file = os.path.join(node11.path, "database/data/default/restore_table/wal.bin") - assert os.path.exists(wal_file) - assert os.path.getsize(wal_file) == 0 + # Check file exists + node11.exec_in_container(['bash', '-c', 'test -f /var/lib/clickhouse/data/default/restore_table/wal.bin']) + # Chech file empty + node11.exec_in_container(['bash', '-c', 'test ! -s /var/lib/clickhouse/data/default/restore_table/wal.bin']) def test_in_memory_deduplication(start_cluster): @@ -509,19 +515,20 @@ def test_in_memory_alters(start_cluster): def test_polymorphic_parts_index(start_cluster): + node1.query('CREATE DATABASE test_index ENGINE=Ordinary') node1.query(''' - CREATE TABLE index_compact(a UInt32, s String) + CREATE TABLE test_index.index_compact(a UInt32, s String) ENGINE = MergeTree ORDER BY a SETTINGS min_rows_for_wide_part = 1000, index_granularity = 128, merge_max_block_size = 100''') - node1.query("INSERT INTO index_compact SELECT number, toString(number) FROM numbers(100)") - node1.query("INSERT INTO index_compact SELECT number, toString(number) FROM numbers(30)") - node1.query("OPTIMIZE TABLE index_compact FINAL") + node1.query("INSERT INTO test_index.index_compact SELECT number, toString(number) FROM numbers(100)") + node1.query("INSERT INTO test_index.index_compact SELECT number, toString(number) FROM numbers(30)") + node1.query("OPTIMIZE TABLE test_index.index_compact FINAL") assert node1.query("SELECT part_type FROM system.parts WHERE table = 'index_compact' AND active") == "Compact\n" assert node1.query("SELECT marks FROM system.parts WHERE table = 'index_compact' AND active") == "2\n" - index_path = os.path.join(node1.path, "database/data/default/index_compact/all_1_2_1/primary.idx") + index_path = os.path.join(node1.path, "database/data/test_index/index_compact/all_1_2_1/primary.idx") f = open(index_path, 'rb') assert os.path.getsize(index_path) == 8 diff --git a/tests/integration/test_quorum_inserts/configs/config.d/remote_servers.xml b/tests/integration/test_quorum_inserts/configs/config.d/remote_servers.xml new file mode 100644 index 00000000000..b1cd417f8b9 --- /dev/null +++ b/tests/integration/test_quorum_inserts/configs/config.d/remote_servers.xml @@ -0,0 +1,20 @@ + + + + + + zero + 9000 + + + first + 9000 + + + second + 9000 + + + + + diff --git a/tests/integration/test_quorum_inserts/test.py b/tests/integration/test_quorum_inserts/test.py index 0adee0afc64..2211333bb26 100644 --- a/tests/integration/test_quorum_inserts/test.py +++ b/tests/integration/test_quorum_inserts/test.py @@ -7,23 +7,21 @@ from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) zero = cluster.add_instance("zero", user_configs=["configs/users.d/settings.xml"], + main_configs=["configs/config.d/remote_servers.xml"], macros={"cluster": "anime", "shard": "0", "replica": "zero"}, with_zookeeper=True) first = cluster.add_instance("first", user_configs=["configs/users.d/settings.xml"], + main_configs=["configs/config.d/remote_servers.xml"], macros={"cluster": "anime", "shard": "0", "replica": "first"}, with_zookeeper=True) second = cluster.add_instance("second", user_configs=["configs/users.d/settings.xml"], + main_configs=["configs/config.d/remote_servers.xml"], macros={"cluster": "anime", "shard": "0", "replica": "second"}, with_zookeeper=True) -def execute_on_all_cluster(query_): - for node in [zero, first, second]: - node.query(query_) - - @pytest.fixture(scope="module") def started_cluster(): global cluster @@ -36,7 +34,7 @@ def started_cluster(): def test_simple_add_replica(started_cluster): - execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") + zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster") create_query = "CREATE TABLE test_simple " \ "(a Int8, d Date) " \ @@ -67,11 +65,11 @@ def test_simple_add_replica(started_cluster): assert '1\t2011-01-01\n' == first.query("SELECT * from test_simple") assert '1\t2011-01-01\n' == second.query("SELECT * from test_simple") - execute_on_all_cluster("DROP TABLE IF EXISTS test_simple") + zero.query("DROP TABLE IF EXISTS test_simple ON CLUSTER cluster") def test_drop_replica_and_achieve_quorum(started_cluster): - execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") + zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster") create_query = "CREATE TABLE test_drop_replica_and_achieve_quorum " \ "(a Int8, d Date) " \ @@ -125,7 +123,7 @@ def test_drop_replica_and_achieve_quorum(started_cluster): assert TSV("1\t2011-01-01\n2\t2012-02-02\n") == TSV( second.query("SELECT * FROM test_drop_replica_and_achieve_quorum ORDER BY a")) - execute_on_all_cluster("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum") + zero.query("DROP TABLE IF EXISTS test_drop_replica_and_achieve_quorum ON CLUSTER cluster") @pytest.mark.parametrize( @@ -136,17 +134,15 @@ def test_drop_replica_and_achieve_quorum(started_cluster): ] ) def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): - execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") + zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster") - create_query = "CREATE TABLE test_quorum_insert_with_drop_partition " \ + create_query = "CREATE TABLE test_quorum_insert_with_drop_partition ON CLUSTER cluster " \ "(a Int8, d Date) " \ - "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "Engine = ReplicatedMergeTree " \ "PARTITION BY d ORDER BY a " print("Create Replicated table with three replicas") zero.query(create_query) - first.query(create_query) - second.query(create_query) print("Stop fetches for test_quorum_insert_with_drop_partition at first replica.") first.query("SYSTEM STOP FETCHES test_quorum_insert_with_drop_partition") @@ -167,9 +163,11 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): print("Sync first replica with others.") first.query("SYSTEM SYNC REPLICA test_quorum_insert_with_drop_partition") - assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ - "where path='/clickhouse/tables/0/test_quorum_insert_with_drop_partition/quorum/last_part' " \ - "format Vertical") + assert "20110101" not in first.query(""" + WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_quorum_insert_with_drop_partition') AS uuid, + '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p + SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical + """) print("Select from updated partition.") if (add_new_data): @@ -179,7 +177,7 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): assert TSV("") == TSV(zero.query("SELECT * FROM test_quorum_insert_with_drop_partition")) assert TSV("") == TSV(second.query("SELECT * FROM test_quorum_insert_with_drop_partition")) - execute_on_all_cluster("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition") + zero.query("DROP TABLE IF EXISTS test_quorum_insert_with_drop_partition ON CLUSTER cluster") @pytest.mark.parametrize( @@ -190,28 +188,24 @@ def test_insert_quorum_with_drop_partition(started_cluster, add_new_data): ] ) def test_insert_quorum_with_move_partition(started_cluster, add_new_data): - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster") - create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source " \ + create_source = "CREATE TABLE test_insert_quorum_with_move_partition_source ON CLUSTER cluster " \ "(a Int8, d Date) " \ - "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "Engine = ReplicatedMergeTree " \ "PARTITION BY d ORDER BY a " - create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination " \ + create_destination = "CREATE TABLE test_insert_quorum_with_move_partition_destination ON CLUSTER cluster " \ "(a Int8, d Date) " \ - "Engine = ReplicatedMergeTree('/clickhouse/tables/{shard}/{table}', '{replica}') " \ + "Engine = ReplicatedMergeTree " \ "PARTITION BY d ORDER BY a " print("Create source Replicated table with three replicas") zero.query(create_source) - first.query(create_source) - second.query(create_source) print("Create destination Replicated table with three replicas") zero.query(create_destination) - first.query(create_destination) - second.query(create_destination) print("Stop fetches for test_insert_quorum_with_move_partition_source at first replica.") first.query("SYSTEM STOP FETCHES test_insert_quorum_with_move_partition_source") @@ -233,9 +227,11 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): print("Sync first replica with others.") first.query("SYSTEM SYNC REPLICA test_insert_quorum_with_move_partition_source") - assert "20110101" not in first.query("SELECT * FROM system.zookeeper " \ - "where path='/clickhouse/tables/0/test_insert_quorum_with_move_partition_source/quorum/last_part' " \ - "format Vertical") + assert "20110101" not in first.query(""" + WITH (SELECT toString(uuid) FROM system.tables WHERE name = 'test_insert_quorum_with_move_partition_source') AS uuid, + '/clickhouse/tables/' || uuid || '/0/quorum/last_part' AS p + SELECT * FROM system.zookeeper WHERE path = p FORMAT Vertical + """) print("Select from updated partition.") if (add_new_data): @@ -246,12 +242,12 @@ def test_insert_quorum_with_move_partition(started_cluster, add_new_data): assert TSV("") == TSV(zero.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) assert TSV("") == TSV(second.query("SELECT * FROM test_insert_quorum_with_move_partition_source")) - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source") - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_source ON CLUSTER cluster") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_move_partition_destination ON CLUSTER cluster") def test_insert_quorum_with_ttl(started_cluster): - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") create_query = "CREATE TABLE test_insert_quorum_with_ttl " \ "(a Int8, d Date) " \ @@ -298,4 +294,4 @@ def test_insert_quorum_with_ttl(started_cluster): assert TSV("2\t2012-02-02\n") == TSV( first.query("SELECT * FROM test_insert_quorum_with_ttl", settings={'select_sequential_consistency': 1})) - execute_on_all_cluster("DROP TABLE IF EXISTS test_insert_quorum_with_ttl") + zero.query("DROP TABLE IF EXISTS test_insert_quorum_with_ttl ON CLUSTER cluster") diff --git a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml index f3b7f959ce9..20b750ffff3 100644 --- a/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_replicated_merge_tree_s3/configs/config.d/storage_conf.xml @@ -22,4 +22,28 @@ 0 + + + + + + node1 + 9000 + + + node2 + 9000 + + + node3 + 9000 + + + + + + + 0 + + diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 1414905759a..4d19793d0b2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -14,11 +14,11 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, + cluster.add_instance("node1", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '1'}, with_minio=True, with_zookeeper=True) - cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, + cluster.add_instance("node2", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '2'}, with_zookeeper=True) - cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'cluster': 'test1'}, + cluster.add_instance("node3", main_configs=["configs/config.d/storage_conf.xml"], macros={'replica': '3'}, with_zookeeper=True) logging.info("Starting cluster...") @@ -49,12 +49,12 @@ def generate_values(date_str, count, sign=1): def create_table(cluster, additional_settings=None): create_table_statement = """ - CREATE TABLE s3_test ( + CREATE TABLE s3_test ON CLUSTER cluster( dt Date, id Int64, data String, INDEX min_max (id) TYPE minmax GRANULARITY 3 - ) ENGINE=ReplicatedMergeTree('/clickhouse/{cluster}/tables/test/s3', '{instance}') + ) ENGINE=ReplicatedMergeTree() PARTITION BY dt ORDER BY (dt, id) SETTINGS storage_policy='s3' @@ -63,8 +63,7 @@ def create_table(cluster, additional_settings=None): create_table_statement += "," create_table_statement += additional_settings - for node in cluster.instances.values(): - node.query(create_table_statement) + cluster.instances.values()[0].query(create_table_statement) @pytest.fixture(autouse=True) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index a407f0b2c7a..c3c86f5a9c5 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -34,7 +34,7 @@ def started_cluster(): for current_node in nodes: current_node.query(''' - CREATE DATABASE mydb ENGINE=Ordinary; + CREATE DATABASE mydb; CREATE TABLE mydb.filtered_table1 (a UInt8, b UInt8) ENGINE MergeTree ORDER BY a; INSERT INTO mydb.filtered_table1 values (0, 0), (0, 1), (1, 0), (1, 1); @@ -360,7 +360,7 @@ def test_miscellaneous_engines(): # ReplicatedCollapsingMergeTree node.query("DROP TABLE mydb.filtered_table1") node.query( - "CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-00/filtered_table1', 'replica1', b) ORDER BY a") + "CREATE TABLE mydb.filtered_table1 (a UInt8, b Int8) ENGINE ReplicatedCollapsingMergeTree('/clickhouse/tables/00-01/filtered_table1', 'replica1', b) ORDER BY a") node.query("INSERT INTO mydb.filtered_table1 values (0, 1), (0, 1), (1, 1), (1, 1)") assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 1], [1, 1]]) diff --git a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh index c3643399ba1..48171b56dd3 100755 --- a/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh +++ b/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh @@ -3,13 +3,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038" -$CLICKHOUSE_CLIENT --query "CREATE DATABASE database_for_dict Engine = Ordinary" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE db_01038" $CLICKHOUSE_CLIENT --query " -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE db_01038.table_for_dict ( key_column UInt64, value Float64 @@ -17,34 +17,34 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column" -$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (1, 1.1)" +$CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (1, 1.1)" $CLICKHOUSE_CLIENT --query " -CREATE DICTIONARY database_for_dict.dict_with_zero_min_lifetime +CREATE DICTIONARY db_01038.dict_with_zero_min_lifetime ( key_column UInt64, value Float64 DEFAULT 77.77 ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'db_01038')) LIFETIME(1) LAYOUT(FLAT())" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))" -$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (2, 2.2)" +$CLICKHOUSE_CLIENT --query "INSERT INTO db_01038.table_for_dict VALUES (2, 2.2)" function check() { - query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))") + query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))") while [ "$query_result" != "2.2" ] do - query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))") + query_result=$($CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))") done } @@ -53,8 +53,8 @@ export -f check; timeout 10 bash -c check -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(1))" -$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('db_01038.dict_with_zero_min_lifetime', 'value', toUInt64(2))" -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS db_01038" diff --git a/tests/queries/0_stateless/01268_dictionary_direct_layout.sql b/tests/queries/0_stateless/01268_dictionary_direct_layout.sql index 9b2f2344242..48642c91102 100644 --- a/tests/queries/0_stateless/01268_dictionary_direct_layout.sql +++ b/tests/queries/0_stateless/01268_dictionary_direct_layout.sql @@ -1,12 +1,12 @@ -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS database_for_dict_01268; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict_01268; -DROP TABLE IF EXISTS database_for_dict.table_for_dict1; -DROP TABLE IF EXISTS database_for_dict.table_for_dict2; -DROP TABLE IF EXISTS database_for_dict.table_for_dict3; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict1; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict2; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict3; -CREATE TABLE database_for_dict.table_for_dict1 +CREATE TABLE database_for_dict_01268.table_for_dict1 ( key_column UInt64, second_column UInt64, @@ -15,9 +15,9 @@ CREATE TABLE database_for_dict.table_for_dict1 ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict1 VALUES (100500, 10000000, 'Hello world'); +INSERT INTO database_for_dict_01268.table_for_dict1 VALUES (100500, 10000000, 'Hello world'); -CREATE TABLE database_for_dict.table_for_dict2 +CREATE TABLE database_for_dict_01268.table_for_dict2 ( region_id UInt64, parent_region UInt64, @@ -26,13 +26,13 @@ CREATE TABLE database_for_dict.table_for_dict2 ENGINE = MergeTree() ORDER BY region_id; -INSERT INTO database_for_dict.table_for_dict2 VALUES (1, 0, 'Russia'); -INSERT INTO database_for_dict.table_for_dict2 VALUES (2, 1, 'Moscow'); -INSERT INTO database_for_dict.table_for_dict2 VALUES (3, 2, 'Center'); -INSERT INTO database_for_dict.table_for_dict2 VALUES (4, 0, 'Great Britain'); -INSERT INTO database_for_dict.table_for_dict2 VALUES (5, 4, 'London'); +INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (1, 0, 'Russia'); +INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (2, 1, 'Moscow'); +INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (3, 2, 'Center'); +INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (4, 0, 'Great Britain'); +INSERT INTO database_for_dict_01268.table_for_dict2 VALUES (5, 4, 'London'); -CREATE TABLE database_for_dict.table_for_dict3 +CREATE TABLE database_for_dict_01268.table_for_dict3 ( region_id UInt64, parent_region Float32, @@ -41,91 +41,91 @@ CREATE TABLE database_for_dict.table_for_dict3 ENGINE = MergeTree() ORDER BY region_id; -INSERT INTO database_for_dict.table_for_dict3 VALUES (1, 0.5, 'Russia'); -INSERT INTO database_for_dict.table_for_dict3 VALUES (2, 1.6, 'Moscow'); -INSERT INTO database_for_dict.table_for_dict3 VALUES (3, 2.3, 'Center'); -INSERT INTO database_for_dict.table_for_dict3 VALUES (4, 0.2, 'Great Britain'); -INSERT INTO database_for_dict.table_for_dict3 VALUES (5, 4.9, 'London'); +INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (1, 0.5, 'Russia'); +INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (2, 1.6, 'Moscow'); +INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (3, 2.3, 'Center'); +INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (4, 0.2, 'Great Britain'); +INSERT INTO database_for_dict_01268.table_for_dict3 VALUES (5, 4.9, 'London'); -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01268; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE db_01268; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; -DROP DICTIONARY IF EXISTS ordinary_db.dict2; -DROP DICTIONARY IF EXISTS ordinary_db.dict3; +DROP DICTIONARY IF EXISTS db_01268.dict1; +DROP DICTIONARY IF EXISTS db_01268.dict2; +DROP DICTIONARY IF EXISTS db_01268.dict3; -CREATE DICTIONARY ordinary_db.dict1 +CREATE DICTIONARY db_01268.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt64 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB 'database_for_dict_01268')) LAYOUT(DIRECT()) SETTINGS(max_result_bytes=1); -CREATE DICTIONARY ordinary_db.dict2 +CREATE DICTIONARY db_01268.dict2 ( region_id UInt64 DEFAULT 0, parent_region UInt64 DEFAULT 0 HIERARCHICAL, region_name String DEFAULT '' ) PRIMARY KEY region_id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB 'database_for_dict_01268')) LAYOUT(DIRECT()); -CREATE DICTIONARY ordinary_db.dict3 +CREATE DICTIONARY db_01268.dict3 ( region_id UInt64 DEFAULT 0, parent_region Float32 DEFAULT 0, region_name String DEFAULT '' ) PRIMARY KEY region_id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict3' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict3' PASSWORD '' DB 'database_for_dict_01268')) LAYOUT(DIRECT()); SELECT 'INITIALIZING DICTIONARY'; -SELECT dictGetHierarchy('ordinary_db.dict2', toUInt64(3)); -SELECT dictHas('ordinary_db.dict2', toUInt64(3)); -SELECT dictHas('ordinary_db.dict2', toUInt64(45)); -SELECT dictIsIn('ordinary_db.dict2', toUInt64(3), toUInt64(1)); -SELECT dictIsIn('ordinary_db.dict2', toUInt64(1), toUInt64(3)); -SELECT dictGetUInt64('ordinary_db.dict2', 'parent_region', toUInt64(3)); -SELECT dictGetUInt64('ordinary_db.dict2', 'parent_region', toUInt64(99)); -SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(3)); -SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(2)); -SELECT dictGetFloat32('ordinary_db.dict3', 'parent_region', toUInt64(1)); -SELECT dictGetString('ordinary_db.dict2', 'region_name', toUInt64(5)); -SELECT dictGetString('ordinary_db.dict2', 'region_name', toUInt64(4)); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(100), 'NONE'); +SELECT dictGetHierarchy('db_01268.dict2', toUInt64(3)); +SELECT dictHas('db_01268.dict2', toUInt64(3)); +SELECT dictHas('db_01268.dict2', toUInt64(45)); +SELECT dictIsIn('db_01268.dict2', toUInt64(3), toUInt64(1)); +SELECT dictIsIn('db_01268.dict2', toUInt64(1), toUInt64(3)); +SELECT dictGetUInt64('db_01268.dict2', 'parent_region', toUInt64(3)); +SELECT dictGetUInt64('db_01268.dict2', 'parent_region', toUInt64(99)); +SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(3)); +SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(2)); +SELECT dictGetFloat32('db_01268.dict3', 'parent_region', toUInt64(1)); +SELECT dictGetString('db_01268.dict2', 'region_name', toUInt64(5)); +SELECT dictGetString('db_01268.dict2', 'region_name', toUInt64(4)); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(100), 'NONE'); -SELECT number + 1, dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(number + 1), 'NONE') chars FROM numbers(10); -SELECT number + 1, dictGetFloat32OrDefault('ordinary_db.dict3', 'parent_region', toUInt64(number + 1), toFloat32(0)) chars FROM numbers(10); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(1), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(2), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(3), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(4), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(5), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(6), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(7), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(8), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(9), 'NONE'); -SELECT dictGetStringOrDefault('ordinary_db.dict2', 'region_name', toUInt64(10), 'NONE'); +SELECT number + 1, dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(number + 1), 'NONE') chars FROM numbers(10); +SELECT number + 1, dictGetFloat32OrDefault('db_01268.dict3', 'parent_region', toUInt64(number + 1), toFloat32(0)) chars FROM numbers(10); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(1), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(2), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(3), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(4), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(5), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(6), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(7), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(8), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(9), 'NONE'); +SELECT dictGetStringOrDefault('db_01268.dict2', 'region_name', toUInt64(10), 'NONE'); -SELECT dictGetUInt64('ordinary_db.dict1', 'second_column', toUInt64(100500)); -- { serverError 396 } +SELECT dictGetUInt64('db_01268.dict1', 'second_column', toUInt64(100500)); -- { serverError 396 } SELECT 'END'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; -DROP DICTIONARY IF EXISTS ordinary_db.dict2; -DROP DICTIONARY IF EXISTS ordinary_db.dict3; +DROP DICTIONARY IF EXISTS db_01268.dict1; +DROP DICTIONARY IF EXISTS db_01268.dict2; +DROP DICTIONARY IF EXISTS db_01268.dict3; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01268; -DROP TABLE IF EXISTS database_for_dict.table_for_dict1; -DROP TABLE IF EXISTS database_for_dict.table_for_dict2; -DROP TABLE IF EXISTS database_for_dict.table_for_dict3; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict1; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict2; +DROP TABLE IF EXISTS database_for_dict_01268.table_for_dict3; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS database_for_dict_01268; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 4ffa2bd74cb..ea256cabdc4 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -89,18 +89,5 @@ "release-build": [ ], "database-atomic": [ - /// Inner tables of materialized views have different names - ///"00738_lock_for_inner_table", - ///"00609_mv_index_in_in", - ///"00510_materizlized_view_and_deduplication_zookeeper", - /// Different database engine - ///"00604_show_create_database", - /// UUID must be specified in ATTACH TABLE - ///"01190_full_attach_syntax", - /// Assumes blocking DROP - ///"01320_create_sync_race_condition", - /// Internal distionary name is different - ///"01225_show_create_table_from_dictionary", - ///"01224_no_superfluous_dict_reload" ] } From 639466afdb94f24f0d43a8a168d78f62a1adf93c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 22 Sep 2020 10:45:08 +0300 Subject: [PATCH 021/131] fix docker images --- docker/test/stateful/run.sh | 2 +- docker/test/stateless_unbundled/run.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 34980508488..616c24395a7 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -27,7 +27,7 @@ ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickho ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY - ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ + ln -s /usr/share/clickhouse-test/config/database_ordinary_usersd.xml /etc/clickhouse-server/config.d/ fi function start() diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index 13ded2290fb..e0338448759 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -44,7 +44,7 @@ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; th ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ fi if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then #FIXME USE_DATABASE_ORDINARY - ln -s /usr/share/clickhouse-test/config/database_ordinary_configd.xml /etc/clickhouse-server/config.d/ + ln -s /usr/share/clickhouse-test/config/database_ordinary_usersd.xml /etc/clickhouse-server/config.d/ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml From d7c7619cd84bea7b6bc8c63b98911e29446c8732 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 22 Sep 2020 13:13:57 +0300 Subject: [PATCH 022/131] add test for https://github.com/ClickHouse/ClickHouse/issues/14570 --- .../0_stateless/01505_log_distributed_deadlock.reference | 0 .../queries/0_stateless/01505_log_distributed_deadlock.sql | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.reference create mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.sql diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.reference b/tests/queries/0_stateless/01505_log_distributed_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql new file mode 100644 index 00000000000..b3f294df327 --- /dev/null +++ b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql @@ -0,0 +1,7 @@ +create table t_local(a int) engine Log; + +create table t_dist (a int) engine Distributed(test_shard_localhost, 'default', 't_local', cityHash64(a)); + +set insert_distributed_sync = 1; + +insert into t_dist values (1); From c9cfdffcd7df7b3c565cf955264b64e72d562a79 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Tue, 22 Sep 2020 16:52:56 +0300 Subject: [PATCH 023/131] mongo uri --- tests/integration/helpers/cluster.py | 4 ++-- .../test_mongo_uri.py | 11 ++--------- 2 files changed, 4 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6d0f038daed..0b7fa9264bd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -486,8 +486,8 @@ class ClickHouseCluster: start = time.time() while time.time() - start < timeout: try: - connection.database_names() - print "Connected to Mongo dbs:", connection.database_names() + connection.list_database_names() + print "Connected to Mongo dbs:", connection.list_database_names() return except Exception as ex: print "Can't connect to Mongo " + str(ex) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 53edfc4bbd2..5c09627d0b9 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -69,14 +69,7 @@ def started_cluster(): finally: cluster.shutdown() -@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +# See comment in SourceMongoURI +@pytest.mark.parametrize("layout_name", ["flat"]) def test_simple(started_cluster, layout_name): simple_tester.execute(layout_name, node) - -@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) -def test_complex(started_cluster, layout_name): - complex_tester.execute(layout_name, node) - -@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) -def test_ranged(started_cluster, layout_name): - ranged_tester.execute(layout_name, node) From 7fcf20e48cb861f5fcf585faad9ee23396a0e29d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 22 Sep 2020 14:56:40 +0300 Subject: [PATCH 024/131] enable more tests --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Storages/StorageDictionary.cpp | 11 ++- src/Storages/StorageDictionary.h | 4 +- .../test_atomic_drop_table/test.py | 2 +- tests/integration/test_backup_restore/test.py | 2 +- .../test.py | 2 +- .../test_cluster_copier/task0_description.xml | 4 +- .../task_month_to_week_description.xml | 4 +- .../task_test_block_size.xml | 4 +- tests/integration/test_cluster_copier/test.py | 21 +++-- .../test_cluster_copier/trivial_test.py | 2 +- .../test_dictionaries_dependency/test.py | 15 ++-- .../test_distributed_format/test.py | 2 +- .../test.py | 2 +- .../test_filesystem_layout/test.py | 16 ++++ tests/integration/test_partition/test.py | 2 +- .../test_polymorphic_parts/test.py | 2 +- tests/integration/test_system_merges/test.py | 2 +- .../01018_ddl_dictionaries_create.reference | 6 +- .../01018_ddl_dictionaries_create.sql | 78 +++++++++---------- .../01018_ddl_dictionaries_select.sql | 2 +- .../01018_ddl_dictionaries_special.sql | 2 +- .../01018_dictionaries_from_dictionaries.sql | 2 +- .../01033_dictionaries_lifetime.sql | 4 +- .../01037_polygon_dicts_correctness_all.sh | 2 +- .../01037_polygon_dicts_correctness_fast.sh | 2 +- .../01037_polygon_dicts_simple_functions.sh | 2 +- ...ionary_invalidate_query_switchover_long.sh | 2 +- .../01041_create_dictionary_if_not_exists.sql | 2 +- ...em_reload_dictionary_reloads_completely.sh | 2 +- ...dictionary_attribute_properties_values.sql | 2 +- .../0_stateless/01048_exists_query.sql | 2 +- .../01053_drop_database_mat_view.sql | 2 +- .../0_stateless/01053_ssd_dictionary.sql | 2 + .../01115_join_with_dictionary.sql | 2 +- .../0_stateless/01190_full_attach_syntax.sql | 2 +- .../01224_no_superfluous_dict_reload.sql | 2 +- ...1225_show_create_table_from_dictionary.sql | 2 +- .../01249_bad_arguments_for_bloom_filter.sql | 2 +- .../01251_dict_is_in_infinite_loop.sql | 2 +- .../01259_dictionary_custom_settings_ddl.sql | 4 +- .../01280_ssd_complex_key_dictionary.sql | 2 + ...20_create_sync_race_condition_zookeeper.sh | 4 +- .../0_stateless/01391_join_on_dict_crash.sql | 2 +- 44 files changed, 130 insertions(+), 107 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 8d67672612c..db2ba9090ec 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -525,7 +525,7 @@ static bool allowDictJoin(StoragePtr joined_storage, const Context & context, St if (!dict) return false; - dict_name = dict->dictionaryName(); + dict_name = dict->resolvedDictionaryName(); auto dictionary = context.getExternalDictionariesLoader().getDictionary(dict_name); if (!dictionary) return false; diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 99645d09d00..e859baa702e 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -92,6 +92,12 @@ String StorageDictionary::generateNamesAndTypesDescription(const NamesAndTypesLi return ss.str(); } +String StorageDictionary::resolvedDictionaryName() const +{ + if (location == Location::SameDatabaseAndNameAsDictionary) + return dictionary_name; + return DatabaseCatalog::instance().resolveDictionaryName(dictionary_name); +} StorageDictionary::StorageDictionary( const StorageID & table_id_, @@ -100,9 +106,6 @@ StorageDictionary::StorageDictionary( Location location_) : IStorage(table_id_) , dictionary_name(dictionary_name_) - , resolved_dictionary_name(location_ == Location::SameDatabaseAndNameAsDictionary - ? dictionary_name - : DatabaseCatalog::instance().resolveDictionaryName(dictionary_name)) , location(location_) { StorageInMemoryMetadata storage_metadata; @@ -135,7 +138,7 @@ Pipe StorageDictionary::read( const size_t max_block_size, const unsigned /*threads*/) { - auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolved_dictionary_name); + auto dictionary = context.getExternalDictionariesLoader().getDictionary(resolvedDictionaryName()); auto stream = dictionary->getBlockInputStream(column_names, max_block_size); /// TODO: update dictionary interface for processors. return Pipe(std::make_shared(stream)); diff --git a/src/Storages/StorageDictionary.h b/src/Storages/StorageDictionary.h index 528ee6533b7..5c7beb88d88 100644 --- a/src/Storages/StorageDictionary.h +++ b/src/Storages/StorageDictionary.h @@ -28,7 +28,8 @@ public: static NamesAndTypesList getNamesAndTypes(const DictionaryStructure & dictionary_structure); static String generateNamesAndTypesDescription(const NamesAndTypesList & list); - const String & dictionaryName() const { return resolved_dictionary_name; } + const String & dictionaryName() const { return dictionary_name; } + String resolvedDictionaryName() const; /// Specifies where the table is located relative to the dictionary. enum class Location @@ -50,7 +51,6 @@ public: private: const String dictionary_name; - const String resolved_dictionary_name; const Location location; protected: diff --git a/tests/integration/test_atomic_drop_table/test.py b/tests/integration/test_atomic_drop_table/test.py index 7ff06c7f369..dc1ad47aa75 100644 --- a/tests/integration/test_atomic_drop_table/test.py +++ b/tests/integration/test_atomic_drop_table/test.py @@ -13,7 +13,7 @@ node1 = cluster.add_instance('node1', main_configs=["configs/config.d/zookeeper_ def start_cluster(): try: cluster.start() - node1.query("CREATE DATABASE zktest ENGINE=Ordinary;") + node1.query("CREATE DATABASE zktest ENGINE=Ordinary;") # Different behaviour with Atomic node1.query( ''' CREATE TABLE zktest.atomic_drop_table (n UInt32) diff --git a/tests/integration/test_backup_restore/test.py b/tests/integration/test_backup_restore/test.py index 111dc6d24f8..170266aaaea 100644 --- a/tests/integration/test_backup_restore/test.py +++ b/tests/integration/test_backup_restore/test.py @@ -14,7 +14,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test ENGINE = Ordinary') + q('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic yield cluster diff --git a/tests/integration/test_backup_with_other_granularity/test.py b/tests/integration/test_backup_with_other_granularity/test.py index 45404c1170c..5ed1cb06787 100644 --- a/tests/integration/test_backup_with_other_granularity/test.py +++ b/tests/integration/test_backup_with_other_granularity/test.py @@ -142,7 +142,7 @@ def test_backup_from_old_version_config(started_cluster): def test_backup_and_alter(started_cluster): - node4.query("CREATE DATABASE test ENGINE=Ordinary") + node4.query("CREATE DATABASE test ENGINE=Ordinary") # Different path in shadow/ with Atomic node4.query("CREATE TABLE test.backup_table(A Int64, B String, C Date) Engine = MergeTree order by tuple()") diff --git a/tests/integration/test_cluster_copier/task0_description.xml b/tests/integration/test_cluster_copier/task0_description.xml index d56053ffd39..72eff8d464d 100644 --- a/tests/integration/test_cluster_copier/task0_description.xml +++ b/tests/integration/test_cluster_copier/task0_description.xml @@ -33,7 +33,7 @@ 3 4 5 6 1 2 0 - ENGINE=ReplicatedMergeTree PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 + ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 d + 1 @@ -93,4 +93,4 @@ - + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/tests/integration/test_cluster_copier/task_month_to_week_description.xml index 26dfc7d3e00..ee134603310 100644 --- a/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -34,7 +34,7 @@ ENGINE= - ReplicatedMergeTree + ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') PARTITION BY toMonday(date) ORDER BY d @@ -97,4 +97,4 @@ - + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/task_test_block_size.xml b/tests/integration/test_cluster_copier/task_test_block_size.xml index c9c99a083ea..ea63d580c1c 100644 --- a/tests/integration/test_cluster_copier/task_test_block_size.xml +++ b/tests/integration/test_cluster_copier/task_test_block_size.xml @@ -28,7 +28,7 @@ ENGINE= - ReplicatedMergeTree + ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/test_block_size', '{replica}') ORDER BY d PARTITION BY partition @@ -99,4 +99,4 @@ - + \ No newline at end of file diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 88dac06f158..2a9e696ca46 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -81,11 +81,11 @@ class Task1: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( cluster_num)) ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + - "ENGINE=ReplicatedMergeTree " + + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " + "PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16") ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)") @@ -110,11 +110,10 @@ class Task1: class Task2: - def __init__(self, cluster, unique_zk_path): + def __init__(self, cluster): self.cluster = cluster self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition" self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read() - self.unique_zk_path = unique_zk_path def start(self): instance = cluster.instances['s0_0_0'] @@ -122,13 +121,11 @@ class Task2: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( cluster_num)) ddl_check_query(instance, - "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) " - "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/" + self.unique_zk_path + "', " - "'{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") + "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") @@ -172,7 +169,7 @@ class Task_test_block_size: ddl_check_query(instance, """ CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64) - ENGINE=ReplicatedMergeTree + ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}') ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2) instance.query( @@ -335,17 +332,17 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster, "test1"), []) + execute_task(Task2(started_cluster), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 035faf0bb9f..3d0c5d0f5b0 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -59,7 +59,7 @@ class TaskTrivial: for node in [source, destination]: node.query("DROP DATABASE IF EXISTS default") - node.query("CREATE DATABASE IF NOT EXISTS default") + node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary") source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " diff --git a/tests/integration/test_dictionaries_dependency/test.py b/tests/integration/test_dictionaries_dependency/test.py index 9c36da229e1..d615f90dc79 100644 --- a/tests/integration/test_dictionaries_dependency/test.py +++ b/tests/integration/test_dictionaries_dependency/test.py @@ -12,15 +12,18 @@ def start_cluster(): try: cluster.start() for node in nodes: - node.query("CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS test") + # Different internal dictionary name with Atomic + node.query("CREATE DATABASE IF NOT EXISTS test_ordinary ENGINE=Ordinary") node.query("CREATE DATABASE IF NOT EXISTS atest") node.query("CREATE DATABASE IF NOT EXISTS ztest") node.query("CREATE TABLE test.source(x UInt64, y UInt64) ENGINE=Log") node.query("INSERT INTO test.source VALUES (5,6)") - node.query("CREATE DICTIONARY test.dict(x UInt64, y UInt64) PRIMARY KEY x " \ - "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) " \ - "LAYOUT(FLAT()) LIFETIME(0)") + for db in ("test", "test_ordinary"): + node.query("CREATE DICTIONARY {}.dict(x UInt64, y UInt64) PRIMARY KEY x " \ + "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'source' DB 'test')) " \ + "LAYOUT(FLAT()) LIFETIME(0)".format(db)) yield cluster finally: @@ -91,10 +94,10 @@ def test_dependency_via_explicit_table(node): def test_dependency_via_dictionary_database(node): node.query("CREATE DATABASE dict_db ENGINE=Dictionary") - d_names = ["test.adict", "test.zdict", "atest.dict", "ztest.dict"] + d_names = ["test_ordinary.adict", "test_ordinary.zdict", "atest.dict", "ztest.dict"] for d_name in d_names: node.query("CREATE DICTIONARY {}(x UInt64, y UInt64) PRIMARY KEY x " \ - "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test.dict' DB 'dict_db')) " \ + "SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'test_ordinary.dict' DB 'dict_db')) " \ "LAYOUT(FLAT()) LIFETIME(0)".format(d_name)) def check(): diff --git a/tests/integration/test_distributed_format/test.py b/tests/integration/test_distributed_format/test.py index 7658814a720..7e9d740c171 100644 --- a/tests/integration/test_distributed_format/test.py +++ b/tests/integration/test_distributed_format/test.py @@ -15,7 +15,7 @@ cluster_param = pytest.mark.parametrize("cluster", [ def started_cluster(): try: cluster.start() - node.query("create database test engine=Ordinary") + node.query("create database test") yield cluster finally: diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index a932e9a55c5..d293b96399d 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -17,7 +17,7 @@ node = cluster.add_instance('node', def start_cluster(): try: cluster.start() - node.query('CREATE DATABASE test ENGINE=Ordinary') + node.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic yield cluster finally: cluster.shutdown() diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index e2441d0d20d..2519d0e5ac3 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -27,3 +27,19 @@ def test_file_path_escaping(started_cluster): node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"]) node.exec_in_container( ["bash", "-c", "test -f /var/lib/clickhouse/shadow/1/data/test/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"]) + +def test_file_path_escaping_atomic_db(started_cluster): + node.query('CREATE DATABASE IF NOT EXISTS `test 2` ENGINE = Atomic') + node.query(''' + CREATE TABLE `test 2`.`T.a_b,l-e!` UUID '12345678-1000-4000-8000-000000000001' (`~Id` UInt32) + ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0; + ''') + node.query('''INSERT INTO `test 2`.`T.a_b,l-e!` VALUES (1);''') + node.query('''ALTER TABLE `test 2`.`T.a_b,l-e!` FREEZE;''') + + node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin"]) + # Check symlink + node.exec_in_container(["bash", "-c", "test -L /var/lib/clickhouse/data/test%202/T%2Ea_b%2Cl%2De%21"]) + node.exec_in_container(["bash", "-c", "test -f /var/lib/clickhouse/data/test%202/T%2Ea_b%2Cl%2De%21/1_1_1_0/%7EId.bin"]) + node.exec_in_container( + ["bash", "-c", "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin"]) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 5b27ff94ddb..679c6fb8c5b 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -13,7 +13,7 @@ path_to_data = '/var/lib/clickhouse/' def started_cluster(): try: cluster.start() - q('CREATE DATABASE test ENGINE = Ordinary') + q('CREATE DATABASE test ENGINE = Ordinary') # Different path in shadow/ with Atomic yield cluster diff --git a/tests/integration/test_polymorphic_parts/test.py b/tests/integration/test_polymorphic_parts/test.py index 39745e4f3f0..50a8192fbc5 100644 --- a/tests/integration/test_polymorphic_parts/test.py +++ b/tests/integration/test_polymorphic_parts/test.py @@ -515,7 +515,7 @@ def test_in_memory_alters(start_cluster): def test_polymorphic_parts_index(start_cluster): - node1.query('CREATE DATABASE test_index ENGINE=Ordinary') + node1.query('CREATE DATABASE test_index ENGINE=Ordinary') # Different paths with Atomic node1.query(''' CREATE TABLE test_index.index_compact(a UInt32, s String) ENGINE = MergeTree ORDER BY a diff --git a/tests/integration/test_system_merges/test.py b/tests/integration/test_system_merges/test.py index 07e6f7331d9..1f2da606cd1 100644 --- a/tests/integration/test_system_merges/test.py +++ b/tests/integration/test_system_merges/test.py @@ -21,7 +21,7 @@ node2 = cluster.add_instance('node2', def started_cluster(): try: cluster.start() - node1.query('CREATE DATABASE test ENGINE=Ordinary') + node1.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic node2.query('CREATE DATABASE test ENGINE=Ordinary') yield cluster diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference index 7c2eca9cedf..5b020911d2e 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference @@ -1,14 +1,14 @@ =DICTIONARY in Ordinary DB -CREATE DICTIONARY ordinary_db.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) dict1 1 -ordinary_db dict1 +db_01018 dict1 ==DETACH DICTIONARY 0 ==ATTACH DICTIONARY dict1 1 -ordinary_db dict1 +db_01018 dict1 ==DROP DICTIONARY 0 =DICTIONARY in Memory DB diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql index d7d7c02baa8..3261b1e61d1 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_create.sql @@ -1,12 +1,12 @@ SET send_logs_level = 'fatal'; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS database_for_dict_01018; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict_01018; -DROP TABLE IF EXISTS database_for_dict.table_for_dict; +DROP TABLE IF EXISTS database_for_dict_01018.table_for_dict; -CREATE TABLE database_for_dict.table_for_dict +CREATE TABLE database_for_dict_01018.table_for_dict ( key_column UInt64, second_column UInt8, @@ -15,64 +15,64 @@ CREATE TABLE database_for_dict.table_for_dict ENGINE = MergeTree() ORDER BY key_column; -INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); +INSERT INTO database_for_dict_01018.table_for_dict VALUES (1, 100, 'Hello world'); -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01018; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE db_01018; SELECT '=DICTIONARY in Ordinary DB'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; +DROP DICTIONARY IF EXISTS db_01018.dict1; -CREATE DICTIONARY ordinary_db.dict1 +CREATE DICTIONARY db_01018.dict1 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SHOW CREATE DICTIONARY ordinary_db.dict1; +SHOW CREATE DICTIONARY db_01018.dict1; -SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; +SHOW DICTIONARIES FROM db_01018 LIKE 'dict1'; -EXISTS DICTIONARY ordinary_db.dict1; +EXISTS DICTIONARY db_01018.dict1; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT '==DETACH DICTIONARY'; -DETACH DICTIONARY ordinary_db.dict1; +DETACH DICTIONARY db_01018.dict1; -SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; +SHOW DICTIONARIES FROM db_01018 LIKE 'dict1'; -EXISTS DICTIONARY ordinary_db.dict1; +EXISTS DICTIONARY db_01018.dict1; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT '==ATTACH DICTIONARY'; -ATTACH DICTIONARY ordinary_db.dict1; +ATTACH DICTIONARY db_01018.dict1; -SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; +SHOW DICTIONARIES FROM db_01018 LIKE 'dict1'; -EXISTS DICTIONARY ordinary_db.dict1; +EXISTS DICTIONARY db_01018.dict1; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; SELECT '==DROP DICTIONARY'; -DROP DICTIONARY IF EXISTS ordinary_db.dict1; +DROP DICTIONARY IF EXISTS db_01018.dict1; -SHOW DICTIONARIES FROM ordinary_db LIKE 'dict1'; +SHOW DICTIONARIES FROM db_01018 LIKE 'dict1'; -EXISTS DICTIONARY ordinary_db.dict1; +EXISTS DICTIONARY db_01018.dict1; SELECT database, name FROM system.dictionaries WHERE name LIKE 'dict1'; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01018; DROP DATABASE IF EXISTS memory_db; @@ -87,7 +87,7 @@ CREATE DICTIONARY memory_db.dict2 third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -- {serverError 48} @@ -112,7 +112,7 @@ CREATE DICTIONARY lazy_db.dict3 third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column, second_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018')) LIFETIME(MIN 1 MAX 10) LAYOUT(COMPLEX_KEY_HASHED()); -- {serverError 48} @@ -120,45 +120,45 @@ DROP DATABASE IF EXISTS lazy_db; SELECT '=DROP DATABASE WITH DICTIONARY'; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01018; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE db_01018; -CREATE DICTIONARY ordinary_db.dict4 +CREATE DICTIONARY db_01018.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SHOW DICTIONARIES FROM ordinary_db; +SHOW DICTIONARIES FROM db_01018; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01018; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE db_01018; -SHOW DICTIONARIES FROM ordinary_db; +SHOW DICTIONARIES FROM db_01018; -CREATE DICTIONARY ordinary_db.dict4 +CREATE DICTIONARY db_01018.dict4 ( key_column UInt64 DEFAULT 0, second_column UInt8 DEFAULT 1, third_column String DEFAULT 'qqq' ) PRIMARY KEY key_column -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict_01018')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT()); -SHOW DICTIONARIES FROM ordinary_db; +SHOW DICTIONARIES FROM db_01018; -DROP DATABASE IF EXISTS ordinary_db; +DROP DATABASE IF EXISTS db_01018; -DROP TABLE IF EXISTS database_for_dict.table_for_dict; +DROP TABLE IF EXISTS database_for_dict_01018.table_for_dict; -DROP DATABASE IF EXISTS database_for_dict; +DROP DATABASE IF EXISTS database_for_dict_01018; DROP DATABASE IF EXISTS memory_db; diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql index f4de269e774..4b548a913ea 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_select.sql @@ -2,7 +2,7 @@ SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; CREATE TABLE database_for_dict.table_for_dict ( diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql index 6d9b499a247..ede5897bdf7 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_special.sql @@ -2,7 +2,7 @@ SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; SELECT '***date dict***'; diff --git a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql index 4d2cd6351b5..86180643f88 100644 --- a/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql +++ b/tests/queries/0_stateless/01018_dictionaries_from_dictionaries.sql @@ -2,7 +2,7 @@ SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; CREATE TABLE database_for_dict.table_for_dict ( diff --git a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 57776e1fec1..0a8288c2df0 100644 --- a/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -2,7 +2,7 @@ SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; DROP TABLE IF EXISTS database_for_dict.table_for_dict; @@ -19,7 +19,7 @@ INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); DROP DATABASE IF EXISTS ordinary_db; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE ordinary_db; DROP DICTIONARY IF EXISTS ordinary_db.dict1; diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh index 1b80fcef80b..e7df8282433 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_all.sh @@ -11,7 +11,7 @@ tar -xf "${CURDIR}"/01037_test_data_search.tar.gz -C "${CURDIR}" $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS test_01037; -CREATE DATABASE test_01037 Engine = Ordinary; +CREATE DATABASE test_01037; DROP TABLE IF EXISTS test_01037.points; CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; " diff --git a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh index 4ca95b72937..22d08d425a6 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_correctness_fast.sh @@ -11,7 +11,7 @@ tar -xf "${CURDIR}"/01037_test_data_perf.tar.gz -C "${CURDIR}" $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS test_01037; -CREATE DATABASE test_01037 Engine = Ordinary; +CREATE DATABASE test_01037; DROP TABLE IF EXISTS test_01037.points; CREATE TABLE test_01037.points (x Float64, y Float64) ENGINE = Memory; " diff --git a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh index d32b75ca735..c3d820e1292 100755 --- a/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh +++ b/tests/queries/0_stateless/01037_polygon_dicts_simple_functions.sh @@ -8,7 +8,7 @@ TMP_DIR="/tmp" $CLICKHOUSE_CLIENT -n --query=" DROP DATABASE IF EXISTS test_01037; -CREATE DATABASE test_01037 Engine = Ordinary; +CREATE DATABASE test_01037; DROP TABLE IF EXISTS test_01037.polygons_array; diff --git a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh index 44a192cf178..6b509ac7925 100755 --- a/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh +++ b/tests/queries/0_stateless/01040_dictionary_invalidate_query_switchover_long.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb" -$CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb Engine = Ordinary" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb" $CLICKHOUSE_CLIENT --query " CREATE TABLE dictdb.dict_invalidate diff --git a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql index 8c30abeb28f..5ec76e6ae91 100644 --- a/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql +++ b/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS dictdb.table_for_dict; DROP DICTIONARY IF EXISTS dictdb.dict_exists; DROP DATABASE IF EXISTS dictdb; -CREATE DATABASE dictdb ENGINE = Ordinary; +CREATE DATABASE dictdb; CREATE TABLE dictdb.table_for_dict ( diff --git a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh index f03f7511a4f..46031a3d508 100755 --- a/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh +++ b/tests/queries/0_stateless/01042_system_reload_dictionary_reloads_completely.sh @@ -8,7 +8,7 @@ set -e -o pipefail # Run the client. $CLICKHOUSE_CLIENT --multiquery <<'EOF' DROP DATABASE IF EXISTS dictdb; -CREATE DATABASE dictdb Engine = Ordinary; +CREATE DATABASE dictdb; CREATE TABLE dictdb.table(x Int64, y Int64, insert_time DateTime) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO dictdb.table VALUES (12, 102, now()); diff --git a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql index afd1c1c5780..adeb5630529 100644 --- a/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql +++ b/tests/queries/0_stateless/01043_dictionary_attribute_properties_values.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS dictdb; -CREATE DATABASE dictdb Engine = Ordinary; +CREATE DATABASE dictdb; CREATE TABLE dictdb.dicttbl(key Int64, value_default String, value_expression String) ENGINE = MergeTree ORDER BY tuple(); INSERT INTO dictdb.dicttbl VALUES (12, 'hello', '55:66:77'); diff --git a/tests/queries/0_stateless/01048_exists_query.sql b/tests/queries/0_stateless/01048_exists_query.sql index 700b4f5983d..31b6d2af6c0 100644 --- a/tests/queries/0_stateless/01048_exists_query.sql +++ b/tests/queries/0_stateless/01048_exists_query.sql @@ -3,7 +3,7 @@ EXISTS TABLE db_01048.t_01048; EXISTS DICTIONARY db_01048.t_01048; DROP DATABASE IF EXISTS db_01048; -CREATE DATABASE db_01048 Engine = Ordinary; +CREATE DATABASE db_01048; DROP TABLE IF EXISTS db_01048.t_01048; EXISTS db_01048.t_01048; diff --git a/tests/queries/0_stateless/01053_drop_database_mat_view.sql b/tests/queries/0_stateless/01053_drop_database_mat_view.sql index 60803bced7e..9f7438d594e 100644 --- a/tests/queries/0_stateless/01053_drop_database_mat_view.sql +++ b/tests/queries/0_stateless/01053_drop_database_mat_view.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS some_tests; -CREATE DATABASE some_tests ENGINE=Ordinary; +CREATE DATABASE some_tests ENGINE=Ordinary; -- Different inner table name with Atomic create table some_tests.my_table ENGINE = MergeTree(day, (day), 8192) as select today() as day, 'mystring' as str; show tables from some_tests; diff --git a/tests/queries/0_stateless/01053_ssd_dictionary.sql b/tests/queries/0_stateless/01053_ssd_dictionary.sql index 416d26bd637..fb4acdeadb4 100644 --- a/tests/queries/0_stateless/01053_ssd_dictionary.sql +++ b/tests/queries/0_stateless/01053_ssd_dictionary.sql @@ -23,6 +23,8 @@ INSERT INTO database_for_dict.table_for_dict SELECT number, 0, -1, 'c' FROM syst DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict; +-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse] +-- Probably we need rewrite it to integration test CREATE DICTIONARY database_for_dict.ssd_dict ( id UInt64, diff --git a/tests/queries/0_stateless/01115_join_with_dictionary.sql b/tests/queries/0_stateless/01115_join_with_dictionary.sql index f1477df7df2..807b53c39c0 100644 --- a/tests/queries/0_stateless/01115_join_with_dictionary.sql +++ b/tests/queries/0_stateless/01115_join_with_dictionary.sql @@ -1,7 +1,7 @@ SET send_logs_level = 'fatal'; DROP DATABASE IF EXISTS db_01115; -CREATE DATABASE db_01115 Engine = Ordinary; +CREATE DATABASE db_01115; USE db_01115; diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.sql b/tests/queries/0_stateless/01190_full_attach_syntax.sql index 225e66bed5c..78f0f53d101 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.sql +++ b/tests/queries/0_stateless/01190_full_attach_syntax.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS test_01190; -CREATE DATABASE test_01190 ENGINE=Ordinary; +CREATE DATABASE test_01190 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01190; CREATE TABLE test_01190.table_for_dict (key UInt64, col UInt8) ENGINE = Memory; diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql index 55689411ad6..da4928a26fb 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01224; DROP DATABASE IF EXISTS dict_db_01224_dictionary; -CREATE DATABASE dict_db_01224 ENGINE=Ordinary; +CREATE DATABASE dict_db_01224 ENGINE=Ordinary; -- Different internal dictionary name with Atomic CREATE DATABASE dict_db_01224_dictionary Engine=Dictionary; CREATE TABLE dict_db_01224.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql index 8980a9fd70b..24d10537dbb 100644 --- a/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql +++ b/tests/queries/0_stateless/01225_show_create_table_from_dictionary.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS dict_db_01225; DROP DATABASE IF EXISTS dict_db_01225_dictionary; -CREATE DATABASE dict_db_01225 ENGINE=Ordinary; +CREATE DATABASE dict_db_01225 ENGINE=Ordinary; -- Different internal dictionary name with Atomic CREATE DATABASE dict_db_01225_dictionary Engine=Dictionary; CREATE TABLE dict_db_01225.dict_data (key UInt64, val UInt64) Engine=Memory(); diff --git a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql index 7bf334582a2..8902b164c09 100644 --- a/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql +++ b/tests/queries/0_stateless/01249_bad_arguments_for_bloom_filter.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS test_01249; -CREATE DATABASE test_01249 ENGINE=Ordinary; +CREATE DATABASE test_01249 ENGINE=Ordinary; -- Full ATTACH requires UUID with Atomic USE test_01249; CREATE TABLE bloom_filter_idx_good(`u64` UInt64, `i32` Int32, `f64` Float64, `d` Decimal(10, 2), `s` String, `e` Enum8('a' = 1, 'b' = 2, 'c' = 3), `dt` Date, INDEX bloom_filter_a i32 TYPE bloom_filter(0, 1) GRANULARITY 1) ENGINE = MergeTree() ORDER BY u64 SETTINGS index_granularity = 8192; -- { serverError 42 } diff --git a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql index decf65dc8cf..8e7e76697b5 100644 --- a/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql +++ b/tests/queries/0_stateless/01251_dict_is_in_infinite_loop.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; DROP TABLE IF EXISTS database_for_dict.dict_source; CREATE TABLE database_for_dict.dict_source (id UInt64, parent_id UInt64, value String) ENGINE = Memory; diff --git a/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql b/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql index cbac234305d..9c2174c8469 100644 --- a/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql +++ b/tests/queries/0_stateless/01259_dictionary_custom_settings_ddl.sql @@ -1,6 +1,6 @@ DROP DATABASE IF EXISTS database_for_dict; -CREATE DATABASE database_for_dict Engine = Ordinary; +CREATE DATABASE database_for_dict; DROP TABLE IF EXISTS database_for_dict.table_for_dict; @@ -17,7 +17,7 @@ INSERT INTO database_for_dict.table_for_dict VALUES (100500, 10000000, 'Hello wo DROP DATABASE IF EXISTS ordinary_db; -CREATE DATABASE ordinary_db ENGINE = Ordinary; +CREATE DATABASE ordinary_db; DROP DICTIONARY IF EXISTS ordinary_db.dict1; diff --git a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql index 952a8c2ff55..9faafb6c0c7 100644 --- a/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql +++ b/tests/queries/0_stateless/01280_ssd_complex_key_dictionary.sql @@ -24,6 +24,8 @@ INSERT INTO database_for_dict.table_for_dict SELECT toString(number), number + 1 DROP DICTIONARY IF EXISTS database_for_dict.ssd_dict; +-- FIXME filesystem error: in create_directory: Permission denied [/var/lib/clickhouse] +-- Probably we need rewrite it to integration test CREATE DICTIONARY database_for_dict.ssd_dict ( k1 String, diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index ce1f08b4c51..f72c9da27ba 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01320" -$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Different bahaviour of DROP with Atomic function thread1() { @@ -26,4 +26,4 @@ timeout 10 bash -c thread2 & wait -$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" 2>&1 | grep -v "New table appeared in database being dropped or detached. Try again." || exit 0 diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.sql b/tests/queries/0_stateless/01391_join_on_dict_crash.sql index 998e0e21745..238a966727f 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.sql +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.sql @@ -1,5 +1,5 @@ DROP DATABASE IF EXISTS db_01391; -CREATE DATABASE db_01391 Engine = Ordinary; +CREATE DATABASE db_01391; USE db_01391; DROP TABLE IF EXISTS t; From f80ffc4dce7166ac7381bc2927021055a10b8dad Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Tue, 22 Sep 2020 18:56:24 +0300 Subject: [PATCH 025/131] restore decimal_casts perf test --- tests/performance/{broken => }/decimal_casts.xml | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename tests/performance/{broken => }/decimal_casts.xml (100%) diff --git a/tests/performance/broken/decimal_casts.xml b/tests/performance/decimal_casts.xml similarity index 100% rename from tests/performance/broken/decimal_casts.xml rename to tests/performance/decimal_casts.xml From cf24789c1b6fe1dc81443cbf1ce6b4b0f10dd97d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 22 Sep 2020 21:25:31 +0300 Subject: [PATCH 026/131] Update 01505_log_distributed_deadlock.sql --- .../0_stateless/01505_log_distributed_deadlock.sql | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql index b3f294df327..2b0b2b97188 100644 --- a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql +++ b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql @@ -1,7 +1,12 @@ -create table t_local(a int) engine Log; +DROP TABLE IF EXISTS t_local; +DROP TABLE IF EXISTS t_dist; -create table t_dist (a int) engine Distributed(test_shard_localhost, 'default', 't_local', cityHash64(a)); +create table t_local(a int) engine Log; +create table t_dist (a int) engine Distributed(test_shard_localhost, currentDatabase(), 't_local', cityHash64(a)); set insert_distributed_sync = 1; insert into t_dist values (1); + +DROP TABLE t_local; +DROP TABLE t_dist; From d2427a121f321ce4978134eb09eb3b32a9bd4606 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 23 Sep 2020 10:15:51 +0300 Subject: [PATCH 027/131] Update decimal_casts.xml --- tests/performance/decimal_casts.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/decimal_casts.xml b/tests/performance/decimal_casts.xml index 6c090faee77..e76a49defce 100644 --- a/tests/performance/decimal_casts.xml +++ b/tests/performance/decimal_casts.xml @@ -1,11 +1,11 @@ - + 10G CREATE TABLE t (x UInt64, d32 Decimal32(3), d64 Decimal64(4), d128 Decimal128(5)) ENGINE = Memory - INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(25000000) SETTINGS max_threads = 8 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(50000000) SETTINGS max_threads = 8 DROP TABLE IF EXISTS t SELECT toUInt32(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null @@ -13,8 +13,8 @@ SELECT toInt64(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null SELECT toUInt64(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null SELECT toInt128(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null - SELECT toInt256(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null - SELECT toUInt256(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null + SELECT toInt256(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t LIMIT 10000000 FORMAT Null + SELECT toUInt256(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t LIMIT 10000000 FORMAT Null SELECT toFloat32(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null SELECT toFloat64(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null From 60bb54093b47743285ff98127841d72ed228b8f8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ya=C4=9F=C4=B1zcan=20De=C4=9Firmenci?= <62724709+ycd@users.noreply.github.com> Date: Wed, 23 Sep 2020 11:18:08 +0300 Subject: [PATCH 028/131] Added Comprehension to Improve Performance --- docs/tools/translate/translate.py | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/docs/tools/translate/translate.py b/docs/tools/translate/translate.py index 6486a8cbcc7..343ab09f12a 100755 --- a/docs/tools/translate/translate.py +++ b/docs/tools/translate/translate.py @@ -49,13 +49,14 @@ def translate_impl(text, target_language=None): def translate(text, target_language=None): - result = [] - for part in re.split(curly_braces_re, text): - if part.startswith('{') and part.endswith('}'): - result.append(part) - else: - result.append(translate_impl(part, target_language=target_language)) - return ''.join(result) + return "".join( + [ + part + if part.startswith("{") and part.endswith("}") + else translate_impl(part, target_language=target_language) + for part in re.split(curly_braces_re, text) + ] + ) def translate_toc(root, lang): From 2b6f4134ebb1759e4facc91ac648f81f923b0383 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 23 Sep 2020 11:21:55 +0300 Subject: [PATCH 029/131] Parallel create/fill in perf test --- docker/test/performance-comparison/compare.sh | 6 ++-- docker/test/performance-comparison/perf.py | 31 +++++++++++-------- docker/test/performance-comparison/report.py | 2 +- 3 files changed, 22 insertions(+), 17 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 9cb337c4fd6..ddcc303da0d 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -726,8 +726,8 @@ create view shortness create table inconsistent_short_marking_report engine File(TSV, 'report/unexpected-query-duration.tsv') as select - multiIf(marked_short and time > 0.1, '"short" queries must run faster than 0.02 s', - not marked_short and time < 0.02, '"normal" queries must run longer than 0.1 s', + multiIf(marked_short and time > 0.1, '\"short\" queries must run faster than 0.02 s', + not marked_short and time < 0.02, '\"normal\" queries must run longer than 0.1 s', '') problem, marked_short, time, test, query_index, query_display_name @@ -1065,7 +1065,7 @@ case "$stage" in # to collect the logs. Prefer not to restart, because addresses might change # and we won't be able to process trace_log data. Start in a subshell, so that # it doesn't interfere with the watchdog through `wait`. - ( get_profiles || restart && get_profiles ) ||: + ( get_profiles || { restart && get_profiles ; } ) ||: # Kill the whole process group, because somehow when the subshell is killed, # the sleep inside remains alive and orphaned. diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 79cdc8ea8d2..2e047e6fb84 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -15,6 +15,7 @@ import sys import time import traceback import xml.etree.ElementTree as et +from threading import Thread from scipy import stats def tsv_escape(s): @@ -157,8 +158,11 @@ for t in tables: print(f'skipped\t{tsv_escape(skipped_message)}') sys.exit(0) -# Run create queries -create_query_templates = [q.text for q in root.findall('create_query')] +# Run create and fill queries. We will run them simultaneously for both servers, +# to save time. +# The weird search is to keep the relative order of elements, which matters, and +# etree doesn't support the appropriate xpath query. +create_query_templates = [q.text for q in root.findall('./*') if q.tag in ('create_query', 'fill_query')] create_queries = substitute_parameters(create_query_templates) # Disallow temporary tables, because the clickhouse_driver reconnects on errors, @@ -170,18 +174,19 @@ for q in create_queries: file = sys.stderr) sys.exit(1) -for conn_index, c in enumerate(all_connections): - for q in create_queries: - c.execute(q) - print(f'create\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') +def do_create(connection, index, queries): + for q in queries: + connection.execute(q) + print(f'create\t{index}\t{connection.last_query.elapsed}\t{tsv_escape(q)}') -# Run fill queries -fill_query_templates = [q.text for q in root.findall('fill_query')] -fill_queries = substitute_parameters(fill_query_templates) -for conn_index, c in enumerate(all_connections): - for q in fill_queries: - c.execute(q) - print(f'fill\t{conn_index}\t{c.last_query.elapsed}\t{tsv_escape(q)}') +threads = [Thread(target = do_create, args = (connection, index, create_queries)) + for index, connection in enumerate(all_connections)] + +for t in threads: + t.start() + +for t in threads: + t.join() # Run the queries in randomized order, but preserve their indexes as specified # in the test XML. To avoid using too much time, limit the number of queries diff --git a/docker/test/performance-comparison/report.py b/docker/test/performance-comparison/report.py index ddd2590eb97..e4096041d69 100755 --- a/docker/test/performance-comparison/report.py +++ b/docker/test/performance-comparison/report.py @@ -476,7 +476,7 @@ if args.report == 'main': text = tableStart('Test Times') text += tableHeader(columns) - allowed_average_run_time = 3.75 # 60 seconds per test at 7 runs + allowed_average_run_time = 1.6 # 30 seconds per test at 7 runs for r in rows: anchor = f'{currentTableAnchor()}.{r[0]}' total_runs = (int(r[7]) + 1) * 2 # one prewarm run, two servers From 59cc997d9c34548bee098fc775ab45c36a21885a Mon Sep 17 00:00:00 2001 From: Ildus Kurbangaliev Date: Wed, 23 Sep 2020 15:12:13 +0200 Subject: [PATCH 030/131] Fix documentation for map functions --- .../functions/tuple-map-functions.md | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index f826b810d23..55f34b5831e 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -9,8 +9,7 @@ toc_title: Working with maps Collect all the keys and sum corresponding values. -Arguments are tuples of two arrays, where items in the first array represent keys, and the second array -contains values for the each key. +Arguments are tuples of two arrays, where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promotable to the one type (Int64, UInt64 or Float64). The common promoted type is used as a type for the result array. @@ -30,8 +29,7 @@ SELECT mapAdd(([toUInt8(1), 2], [1, 1]), ([toUInt8(1), 2], [1, 1])) as res, toTy Collect all the keys and subtract corresponding values. -Arguments are tuples of two arrays, where items in the first array represent keys, and the second array -contains values for the each key. +Arguments are tuples of two arrays, where items in the first array represent keys, and the second array contains values for the each key. All key arrays should have same type, and all value arrays should contain items which are promotable to the one type (Int64, UInt64 or Float64). The common promoted type is used as a type for the result array. @@ -45,25 +43,24 @@ SELECT mapSubtract(([toUInt8(1), 2], [toInt32(1), 1]), ([toUInt8(1), 2], [toInt3 ┌─res────────────┬─type──────────────────────────────┐ │ ([1,2],[-1,0]) │ Tuple(Array(UInt8), Array(Int64)) │ └────────────────┴───────────────────────────────────┘ -```` +``` ## mapPopulateSeries {#function-mappopulateseries} Syntax: `mapPopulateSeries((keys : Array(), values : Array()[, max : ])` -Generates a map, where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from `keys` array with step size of one, -and corresponding values taken from `values` array. If the value is not specified for the key, then it uses default value in the resulting map. +Generates a map, where keys are a series of numbers, from minimum to maximum keys (or `max` argument if it specified) taken from `keys` array with step size of one, and corresponding values taken from `values` array. If the value is not specified for the key, then it uses default value in the resulting map. For repeated keys only the first value (in order of appearing) gets associated with the key. The number of elements in `keys` and `values` must be the same for each row. Returns a tuple of two arrays: keys in sorted order, and values the corresponding keys. -``` sql +```sql select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type; ``` -``` text +```text ┌─res──────────────────────────┬─type──────────────────────────────┐ │ ([1,2,3,4,5],[11,22,0,44,0]) │ Tuple(Array(UInt8), Array(UInt8)) │ └──────────────────────────────┴───────────────────────────────────┘ From 9f137e77e781c7f1a571fafe7ba293eed1ce6077 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Wed, 23 Sep 2020 18:41:52 +0300 Subject: [PATCH 031/131] Update decimal_casts.xml --- tests/performance/decimal_casts.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/decimal_casts.xml b/tests/performance/decimal_casts.xml index e76a49defce..7f609c8bd9d 100644 --- a/tests/performance/decimal_casts.xml +++ b/tests/performance/decimal_casts.xml @@ -1,11 +1,11 @@ - 10G + 15G CREATE TABLE t (x UInt64, d32 Decimal32(3), d64 Decimal64(4), d128 Decimal128(5)) ENGINE = Memory - INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(50000000) SETTINGS max_threads = 8 + INSERT INTO t SELECT number AS x, x % 1000000 AS d32, x AS d64, x d128 FROM numbers_mt(100000000) SETTINGS max_threads = 8 DROP TABLE IF EXISTS t SELECT toUInt32(x) y, toDecimal32(y, 1), toDecimal64(y, 5), toDecimal128(y, 6) FROM t FORMAT Null From a8266a3d3ca99c6c4dc22d219a293deccc759609 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 23 Sep 2020 21:28:59 +0300 Subject: [PATCH 032/131] fix DDDL with cross-replication and Atomic --- programs/copier/ClusterCopier.cpp | 4 +- src/Common/Macros.cpp | 54 +++++++++++++------ src/Common/Macros.h | 21 ++++++-- src/Interpreters/Cluster.cpp | 17 ++++++ src/Interpreters/Cluster.h | 4 ++ src/Interpreters/InterpreterCreateQuery.cpp | 54 +++++++++++++++++-- src/Interpreters/InterpreterCreateQuery.h | 2 + .../test_cluster_copier/task0_description.xml | 4 +- .../task_month_to_week_description.xml | 4 +- .../task_test_block_size.xml | 4 +- tests/integration/test_cluster_copier/test.py | 21 ++++---- .../test_cluster_copier/trivial_test.py | 2 +- .../integration/test_distributed_ddl/test.py | 15 +++++- .../test.py | 27 ++++++++++ 14 files changed, 192 insertions(+), 41 deletions(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index b3d1ca7bcec..4ee14b14119 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -1477,7 +1477,9 @@ TaskStatus ClusterCopier::processPartitionPieceTaskImpl( { auto create_query_push_ast = rewriteCreateQueryStorage(task_shard.current_pull_table_create_query, task_table.table_push, task_table.engine_push_ast); - create_query_push_ast->as().if_not_exists = true; + auto & create = create_query_push_ast->as(); + create.if_not_exists = true; + InterpreterCreateQuery::prepareOnClusterQuery(create, context, task_table.cluster_push_name); String query = queryToString(create_query_push_ast); LOG_DEBUG(log, "Create destination tables. Query: {}", query); diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index a4981fa5be3..e3735c44359 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -23,18 +23,15 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String & } String Macros::expand(const String & s, - size_t level, - const String & database_name, - const String & table_name, - const UUID & uuid) const + MacroExpansionInfo & info) const { if (s.find('{') == String::npos) return s; - if (level && s.size() > 65536) + if (info.level && s.size() > 65536) throw Exception("Too long string while expanding macros", ErrorCodes::SYNTAX_ERROR); - if (level >= 10) + if (info.level >= 10) throw Exception("Too deep recursion while expanding macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR); String res; @@ -64,17 +61,28 @@ String Macros::expand(const String & s, /// Prefer explicit macros over implicit. if (it != macros.end()) res += it->second; - else if (macro_name == "database" && !database_name.empty()) - res += database_name; - else if (macro_name == "table" && !table_name.empty()) - res += table_name; + else if (macro_name == "database" && !info.database_name.empty()) + res += info.database_name; + else if (macro_name == "table" && !info.table_name.empty()) + res += info.table_name; else if (macro_name == "uuid") { - if (uuid == UUIDHelpers::Nil) + if (info.uuid == UUIDHelpers::Nil) throw Exception("Macro 'uuid' and empty arguments of ReplicatedMergeTree " "are supported only for ON CLUSTER queries with Atomic database engine", ErrorCodes::SYNTAX_ERROR); - res += toString(uuid); + /// For ON CLUSTER queries we don't want to require all macros definitions in initiator's config. + /// However, initiator must check that for cross-replication cluster zookeeper_path does not contain {uuid} macro. + /// It becomes impossible to check if {uuid} is contained inside some unknown macro. + if (info.level) + throw Exception("Macro 'uuid' should not be inside another macro", ErrorCodes::SYNTAX_ERROR); + res += toString(info.uuid); + info.expanded_uuid = true; + } + else if (info.ignore_unknown) + { + res += macro_name; + info.has_unknown = true; } else throw Exception("No macro '" + macro_name + @@ -84,7 +92,8 @@ String Macros::expand(const String & s, pos = end + 1; } - return expand(res, level + 1, database_name, table_name); + ++info.level; + return expand(res, info); } String Macros::getValue(const String & key) const @@ -94,9 +103,20 @@ String Macros::getValue(const String & key) const throw Exception("No macro " + key + " in config", ErrorCodes::SYNTAX_ERROR); } + +String Macros::expand(const String & s) const +{ + MacroExpansionInfo info; + return expand(s, info); +} + String Macros::expand(const String & s, const StorageID & table_id, bool allow_uuid) const { - return expand(s, 0, table_id.database_name, table_id.table_name, allow_uuid ? table_id.uuid : UUIDHelpers::Nil); + MacroExpansionInfo info; + info.database_name = table_id.database_name; + info.table_name = table_id.table_name; + info.uuid = allow_uuid ? table_id.uuid : UUIDHelpers::Nil; + return expand(s, info); } Names Macros::expand(const Names & source_names, size_t level) const @@ -104,8 +124,12 @@ Names Macros::expand(const Names & source_names, size_t level) const Names result_names; result_names.reserve(source_names.size()); + MacroExpansionInfo info; for (const String & name : source_names) - result_names.push_back(expand(name, level)); + { + info.level = level; + result_names.push_back(expand(name, info)); + } return result_names; } diff --git a/src/Common/Macros.h b/src/Common/Macros.h index bcd6075782e..6e4f25d55ef 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -27,15 +27,28 @@ public: Macros() = default; Macros(const Poco::Util::AbstractConfiguration & config, const String & key); + struct MacroExpansionInfo + { + /// Settings + String database_name; + String table_name; + UUID uuid = UUIDHelpers::Nil; + bool ignore_unknown = false; + + /// Information about macro expansion + size_t level = 0; + bool expanded_uuid = false; + bool has_unknown = false; + }; + /** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file. * If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively. * level - the level of recursion. */ String expand(const String & s, - size_t level = 0, - const String & database_name = "", - const String & table_name = "", - const UUID & uuid = UUIDHelpers::Nil) const; + MacroExpansionInfo & info) const; + + String expand(const String & s) const; String expand(const String & s, const StorageID & table_id, bool allow_uuid) const; diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index b385e74adc5..8a98e8282a6 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -623,4 +623,21 @@ const std::string & Cluster::ShardInfo::pathForInsert(bool prefer_localhost_repl return dir_name_for_internal_replication_with_local; } +bool Cluster::maybeCrossReplication() const +{ + /// Cluster can be used for cross-replication if some replicas have different default database names, + /// so one clickhouse-server instance can contain multiple replicas. + + if (addresses_with_failover.empty()) + return false; + + const String & database_name = addresses_with_failover.front().front().default_database; + for (const auto & shard : addresses_with_failover) + for (const auto & replica : shard) + if (replica.default_database != database_name) + return true; + + return false; +} + } diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 4985c70e6e2..c8225c81453 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -193,6 +193,10 @@ public: /// Get a new Cluster that contains all servers (all shards with all replicas) from existing cluster as independent shards. std::unique_ptr getClusterWithReplicasAsShards(const Settings & settings) const; + /// Returns false if cluster configuration doesn't allow to use it for cross-replication. + /// NOTE: true does not mean, that it's actually a cross-replication cluster. + bool maybeCrossReplication() const; + private: using SlotToShard = std::vector; SlotToShard slot_to_shard; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index cc39d6fc8a3..6f318b3658a 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -853,15 +854,60 @@ BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create) return {}; } +void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, const Context & context, const String & cluster_name) +{ + if (create.attach) + return; + + /// For CREATE query generate UUID on initiator, so it will be the same on all hosts. + /// It will be ignored if database does not support UUIDs. + if (create.uuid == UUIDHelpers::Nil) + create.uuid = UUIDHelpers::generateV4(); + + /// For cross-replication cluster we cannot use UUID in replica path. + String cluster_name_expanded = context.getMacros()->expand(cluster_name); + ClusterPtr cluster = context.getCluster(cluster_name_expanded); + + if (cluster->maybeCrossReplication()) + { + /// Check that {uuid} macro is not used in zookeeper_path for ReplicatedMergeTree. + /// Otherwise replicas will generate different paths. + if (!create.storage) + return; + if (!create.storage->engine) + return; + if (!startsWith(create.storage->engine->name, "Replicated")) + return; + + bool has_explicit_zk_path_arg = create.storage->engine->arguments && + create.storage->engine->arguments->children.size() >= 2 && + create.storage->engine->arguments->children[0]->as() && + create.storage->engine->arguments->children[0]->as()->value.getType() == Field::Types::String; + + if (has_explicit_zk_path_arg) + { + String zk_path = create.storage->engine->arguments->children[0]->as()->value.get(); + Macros::MacroExpansionInfo info; + info.uuid = create.uuid; + info.ignore_unknown = true; + context.getMacros()->expand(zk_path, info); + if (!info.expanded_uuid) + return; + } + + throw Exception("Seems like cluster is configured for cross-replication, " + "but zookeeper_path for ReplicatedMergeTree is not specified or contains {uuid} macro. " + "It's not supported for cross replication, because tables must have different UUIDs. " + "Please specify unique zookeeper_path explicitly.", ErrorCodes::INCORRECT_QUERY); + } +} + BlockIO InterpreterCreateQuery::execute() { auto & create = query_ptr->as(); if (!create.cluster.empty()) { - /// For CREATE query generate UUID on initiator, so it will be the same on all hosts. - /// It will be ignored if database does not support UUIDs. - if (!create.attach && create.uuid == UUIDHelpers::Nil) - create.uuid = UUIDHelpers::generateV4(); + prepareOnClusterQuery(create, context, create.cluster); return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess()); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 4a5d57c11d1..07fca5f3910 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -55,6 +55,8 @@ public: static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context, bool sanity_check_compression_codecs); static ConstraintsDescription getConstraintsDescription(const ASTExpressionList * constraints); + static void prepareOnClusterQuery(ASTCreateQuery & create, const Context & context, const String & cluster_name); + private: struct TableProperties { diff --git a/tests/integration/test_cluster_copier/task0_description.xml b/tests/integration/test_cluster_copier/task0_description.xml index 72eff8d464d..d56053ffd39 100644 --- a/tests/integration/test_cluster_copier/task0_description.xml +++ b/tests/integration/test_cluster_copier/task0_description.xml @@ -33,7 +33,7 @@ 3 4 5 6 1 2 0 - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/hits', '{replica}') PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 + ENGINE=ReplicatedMergeTree PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16 d + 1 @@ -93,4 +93,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_month_to_week_description.xml b/tests/integration/test_cluster_copier/task_month_to_week_description.xml index ee134603310..26dfc7d3e00 100644 --- a/tests/integration/test_cluster_copier/task_month_to_week_description.xml +++ b/tests/integration/test_cluster_copier/task_month_to_week_description.xml @@ -34,7 +34,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/b', '{replica}') + ReplicatedMergeTree PARTITION BY toMonday(date) ORDER BY d @@ -97,4 +97,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/task_test_block_size.xml b/tests/integration/test_cluster_copier/task_test_block_size.xml index ea63d580c1c..c9c99a083ea 100644 --- a/tests/integration/test_cluster_copier/task_test_block_size.xml +++ b/tests/integration/test_cluster_copier/task_test_block_size.xml @@ -28,7 +28,7 @@ ENGINE= - ReplicatedMergeTree('/clickhouse/tables/cluster{cluster}/{shard}/test_block_size', '{replica}') + ReplicatedMergeTree ORDER BY d PARTITION BY partition @@ -99,4 +99,4 @@ - \ No newline at end of file + diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index 2a9e696ca46..88dac06f158 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -81,11 +81,11 @@ class Task1: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, "CREATE TABLE hits ON CLUSTER cluster0 (d UInt64, d1 UInt64 MATERIALIZED d+1) " + - "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/hits', '{replica}') " + + "ENGINE=ReplicatedMergeTree " + "PARTITION BY d % 3 ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d) SETTINGS index_granularity = 16") ddl_check_query(instance, "CREATE TABLE hits_all ON CLUSTER cluster0 (d UInt64) ENGINE=Distributed(cluster0, default, hits, d)") @@ -110,10 +110,11 @@ class Task1: class Task2: - def __init__(self, cluster): + def __init__(self, cluster, unique_zk_path): self.cluster = cluster self.zk_task_path = "/clickhouse-copier/task_month_to_week_partition" self.copier_task_config = open(os.path.join(CURRENT_TEST_DIR, 'task_month_to_week_description.xml'), 'r').read() + self.unique_zk_path = unique_zk_path def start(self): instance = cluster.instances['s0_0_0'] @@ -121,11 +122,13 @@ class Task2: for cluster_num in ["0", "1"]: ddl_check_query(instance, "DROP DATABASE IF EXISTS default ON CLUSTER cluster{}".format(cluster_num)) ddl_check_query(instance, - "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{} ENGINE=Ordinary".format( + "CREATE DATABASE IF NOT EXISTS default ON CLUSTER cluster{}".format( cluster_num)) ddl_check_query(instance, - "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/a', '{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") + "CREATE TABLE a ON CLUSTER cluster0 (date Date, d UInt64, d1 UInt64 ALIAS d+1) " + "ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/" + self.unique_zk_path + "', " + "'{replica}', date, intHash64(d), (date, intHash64(d)), 8192)") ddl_check_query(instance, "CREATE TABLE a_all ON CLUSTER cluster0 (date Date, d UInt64) ENGINE=Distributed(cluster0, default, a, d)") @@ -169,7 +172,7 @@ class Task_test_block_size: ddl_check_query(instance, """ CREATE TABLE test_block_size ON CLUSTER shard_0_0 (partition Date, d UInt64) - ENGINE=ReplicatedMergeTree('/clickhouse/tables/cluster_{cluster}/{shard}/test_block_size', '{replica}') + ENGINE=ReplicatedMergeTree ORDER BY (d, sipHash64(d)) SAMPLE BY sipHash64(d)""", 2) instance.query( @@ -332,17 +335,17 @@ def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offs @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster), []) + execute_task(Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 3d0c5d0f5b0..035faf0bb9f 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -59,7 +59,7 @@ class TaskTrivial: for node in [source, destination]: node.query("DROP DATABASE IF EXISTS default") - node.query("CREATE DATABASE IF NOT EXISTS default ENGINE=Ordinary") + node.query("CREATE DATABASE IF NOT EXISTS default") source.query("CREATE TABLE trivial (d UInt64, d1 UInt64 MATERIALIZED d+1) " "ENGINE=ReplicatedMergeTree('/clickhouse/tables/source_trivial_cluster/1/trivial', '1') " diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 7f254d03015..b788dafe167 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -327,6 +327,8 @@ def test_replicated_without_arguments(test_cluster): rules = test_cluster.pm_random_drops.pop_rules() instance = test_cluster.instances['ch1'] test_cluster.ddl_check_query(instance, "CREATE DATABASE test_atomic ON CLUSTER cluster ENGINE=Atomic") + assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ + instance.query_and_get_error("CREATE TABLE test_atomic.rmt (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "DROP TABLE test_atomic.rmt ON CLUSTER cluster") @@ -334,9 +336,20 @@ def test_replicated_without_arguments(test_cluster): "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") test_cluster.ddl_check_query(instance, "RENAME TABLE test_atomic.rmt TO test_atomic.rmt_renamed ON CLUSTER cluster") test_cluster.ddl_check_query(instance, - "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + "CREATE TABLE test_atomic.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/clickhouse/tables/{uuid}/{shard}', '{replica}') ORDER BY n") test_cluster.ddl_check_query(instance, "EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster") + assert instance.query("SELECT countDistinct(uuid) from clusterAllReplicas('cluster', 'system', 'databases') WHERE uuid != 0 AND name='test_atomic'") == "1\n" + assert instance.query("SELECT countDistinct(uuid) from clusterAllReplicas('cluster', 'system', 'tables') WHERE uuid != 0 AND name='rmt'") == "1\n" + test_cluster.ddl_check_query(instance, "DROP DATABASE test_atomic ON CLUSTER cluster") + + test_cluster.ddl_check_query(instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary") + assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ + instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + assert "are supported only for ON CLUSTER queries with Atomic database engine" in \ + instance.query_and_get_error("CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n") + test_cluster.ddl_check_query(instance, "CREATE TABLE test_ordinary.rmt ON CLUSTER cluster (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{table}/', '{replica}') ORDER BY n") + test_cluster.ddl_check_query(instance, "DROP DATABASE test_ordinary ON CLUSTER cluster") test_cluster.pm_random_drops.push_rules(rules) diff --git a/tests/integration/test_distributed_ddl_on_cross_replication/test.py b/tests/integration/test_distributed_ddl_on_cross_replication/test.py index 16238f0326d..85800b2e5e6 100644 --- a/tests/integration/test_distributed_ddl_on_cross_replication/test.py +++ b/tests/integration/test_distributed_ddl_on_cross_replication/test.py @@ -77,3 +77,30 @@ def test_alter_ddl(started_cluster): node2.query("SYSTEM SYNC REPLICA replica_2.replicated_local;", timeout=5) assert_eq_with_retry(node1, "SELECT count(*) FROM replica_2.replicated", '0') + +def test_atomic_database(started_cluster): + node1.query('''DROP DATABASE IF EXISTS replica_1 ON CLUSTER cross_3shards_2replicas; + DROP DATABASE IF EXISTS replica_2 ON CLUSTER cross_3shards_2replicas; + CREATE DATABASE replica_1 ON CLUSTER cross_3shards_2replicas ENGINE=Atomic; + CREATE DATABASE replica_2 ON CLUSTER cross_3shards_2replicas ENGINE=Atomic;''') + + assert "It's not supported for cross replication" in \ + node1.query_and_get_error("CREATE TABLE rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + assert "It's not supported for cross replication" in \ + node1.query_and_get_error("CREATE TABLE replica_1.rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree ORDER BY n") + assert "It's not supported for cross replication" in \ + node1.query_and_get_error("CREATE TABLE rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n") + assert "It's not supported for cross replication" in \ + node1.query_and_get_error("CREATE TABLE replica_2.rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree('/{shard}/{uuid}/', '{replica}') ORDER BY n") + assert "For a distributed DDL on circular replicated cluster its table name must be qualified by database name" in \ + node1.query_and_get_error("CREATE TABLE rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree('/tables/{shard}/rmt/', '{replica}') ORDER BY n") + + node1.query("CREATE TABLE replica_1.rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree('/tables/{shard}/rmt/', '{replica}') ORDER BY n") + node1.query("CREATE TABLE replica_2.rmt ON CLUSTER cross_3shards_2replicas (n UInt64, s String) ENGINE=ReplicatedMergeTree('/tables/{shard_bk}/rmt/', '{replica_bk}') ORDER BY n") + + assert node1.query("SELECT countDistinct(uuid) from remote('node1,node2,node3', 'system', 'databases') WHERE uuid != 0 AND name='replica_1'") == "1\n" + assert node1.query("SELECT countDistinct(uuid) from remote('node1,node2,node3', 'system', 'tables') WHERE uuid != 0 AND name='rmt'") == "2\n" + + node1.query("INSERT INTO replica_1.rmt VALUES (1, 'test')") + node2.query("SYSTEM SYNC REPLICA replica_2.rmt", timeout=5) + assert_eq_with_retry(node2, "SELECT * FROM replica_2.rmt", '1\ttest') From 2d4bdcfec0e2982f102a72b6f7fd40acd197e421 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 23 Sep 2020 22:31:47 +0300 Subject: [PATCH 033/131] file and executables --- tests/integration/helpers/external_sources.py | 6 +++--- .../test_executable_cache.py | 10 +++------- .../test_executable_hashed.py | 8 ++++---- .../test_file.py | 6 +++--- 4 files changed, 13 insertions(+), 17 deletions(-) diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 0d01a1bcbfd..a52cf7a02d8 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -333,16 +333,16 @@ class _SourceExecutableBase(ExternalSource): user='root') -class SourceExecutableCache(_SourceExecutableBase): +class SourceExecutableHashed(_SourceExecutableBase): def _get_cmd(self, path): return "cat {}".format(path) def compatible_with_layout(self, layout): - return 'cache' not in layout.name + return 'hashed' in layout.name -class SourceExecutableHashed(_SourceExecutableBase): +class SourceExecutableCache(_SourceExecutableBase): def _get_cmd(self, path): return "cat - >/dev/null;cat {}".format(path) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py index 87b73d3acc1..1d741d5271c 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -46,7 +46,7 @@ def setup_module(module): for fname in os.listdir(DICT_CONFIG_PATH): dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): @@ -69,14 +69,10 @@ def started_cluster(): finally: cluster.shutdown() -@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +@pytest.mark.parametrize("layout_name", ['cache']) def test_simple(started_cluster, layout_name): simple_tester.execute(layout_name, node) -@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +@pytest.mark.parametrize("layout_name", ['complex_key_cache']) def test_complex(started_cluster, layout_name): complex_tester.execute(layout_name, node) - -@pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) -def test_ranged(started_cluster, layout_name): - ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py index 5ab5ee545e7..03af42bb1d4 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -46,7 +46,7 @@ def setup_module(module): for fname in os.listdir(DICT_CONFIG_PATH): dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): @@ -69,14 +69,14 @@ def started_cluster(): finally: cluster.shutdown() -@pytest.mark.parametrize("layout_name", LAYOUTS_SIMPLE) +@pytest.mark.parametrize("layout_name", ['hashed']) def test_simple(started_cluster, layout_name): simple_tester.execute(layout_name, node) -@pytest.mark.parametrize("layout_name", LAYOUTS_COMPLEX) +@pytest.mark.parametrize("layout_name", ['complex_key_hashed']) def test_complex(started_cluster, layout_name): complex_tester.execute(layout_name, node) - + @pytest.mark.parametrize("layout_name", LAYOUTS_RANGED) def test_ranged(started_cluster, layout_name): ranged_tester.execute(layout_name, node) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py index 893d095a143..f786bda847f 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -46,7 +46,7 @@ def setup_module(module): for fname in os.listdir(DICT_CONFIG_PATH): dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): @@ -69,11 +69,11 @@ def started_cluster(): finally: cluster.shutdown() -@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_SIMPLE).difference(set("cache, direct"))) ) +@pytest.mark.parametrize("layout_name", set(LAYOUTS_SIMPLE).difference({'cache', 'direct'}) ) def test_simple(started_cluster, layout_name): simple_tester.execute(layout_name, node) -@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_SIMPLE).difference(set("complex_key_cache, complex_key_direct")))) +@pytest.mark.parametrize("layout_name", list(set(LAYOUTS_COMPLEX).difference({'complex_key_cache', 'complex_key_direct'}))) def test_complex(started_cluster, layout_name): complex_tester.execute(layout_name, node) From c64ac8596c13eb9b2e9555531c1e89407820a26d Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 23 Sep 2020 22:36:47 +0300 Subject: [PATCH 034/131] better --- .../test_clickhouse_local.py | 2 +- .../test_clickhouse_remote.py | 2 +- .../test_dictionaries_all_layouts_separate_sources/test_http.py | 2 +- .../test_https.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py index c8597708c04..1adc02ba6aa 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -46,7 +46,7 @@ def setup_module(module): for fname in os.listdir(DICT_CONFIG_PATH): dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py index 4f190e4c332..4e7f307b959 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -48,7 +48,7 @@ def setup_module(module): cluster.add_instance('clickhouse1', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py index 8c7519fd0a7..80baee5ee45 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -48,7 +48,7 @@ def setup_module(module): cluster.add_instance('clickhouse1', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py index c5280aa0076..ccac2cfd268 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -48,7 +48,7 @@ def setup_module(module): cluster.add_instance('clickhouse1', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): From b2d66770ee80d1c249ac2726b6820b86616ee558 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Sep 2020 21:35:29 +0300 Subject: [PATCH 035/131] Update protobuf to the latest version. --- contrib/protobuf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/protobuf b/contrib/protobuf index d6a10dd3db5..445d1ae73a4 160000 --- a/contrib/protobuf +++ b/contrib/protobuf @@ -1 +1 @@ -Subproject commit d6a10dd3db55d8f7f9e464db9151874cde1f79ec +Subproject commit 445d1ae73a450b1e94622e7040989aa2048402e3 From ba3257de22ab6ccb657c5d10c75698a82306fbb9 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 23 Sep 2020 23:20:59 +0300 Subject: [PATCH 036/131] Disable warnings in protobuf's headers. --- contrib/protobuf-cmake/CMakeLists.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/contrib/protobuf-cmake/CMakeLists.txt b/contrib/protobuf-cmake/CMakeLists.txt index 683429194fc..1f8d9b02b3e 100644 --- a/contrib/protobuf-cmake/CMakeLists.txt +++ b/contrib/protobuf-cmake/CMakeLists.txt @@ -11,3 +11,7 @@ else () endif () add_subdirectory("${protobuf_SOURCE_DIR}/cmake" "${protobuf_BINARY_DIR}") + +# We don't want to stop compilation on warnings in protobuf's headers. +# The following line overrides the value assigned by the command target_include_directories() in libprotobuf.cmake +set_property(TARGET libprotobuf PROPERTY INTERFACE_SYSTEM_INCLUDE_DIRECTORIES ${protobuf_SOURCE_DIR}/src) From 5964e242da081443f0fad942787560751522ad67 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Sep 2020 00:48:54 +0300 Subject: [PATCH 037/131] trigger CI From a85bb16c66f0c795290cf8dc740c32c4d3ba0493 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 10:33:22 +0300 Subject: [PATCH 038/131] Remove zookeeper from CI images --- docker/test/stateless/Dockerfile | 4 +--- docker/test/stateless/run.sh | 2 -- docker/test/stateless_unbundled/Dockerfile | 2 -- docker/test/stateless_unbundled/run.sh | 2 -- docker/test/stateless_with_coverage/Dockerfile | 2 -- docker/test/stateless_with_coverage/run.sh | 3 --- 6 files changed, 1 insertion(+), 14 deletions(-) diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 409a1b07bef..516d8d5842b 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -21,9 +21,7 @@ RUN apt-get update -y \ telnet \ tree \ unixodbc \ - wget \ - zookeeper \ - zookeeperd + wget RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index b6b48cd0943..adec0f51490 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -48,8 +48,6 @@ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml -service zookeeper start -sleep 5 service clickhouse-server start && sleep 5 if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index b05e46406da..dbb7ae2477f 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -67,8 +67,6 @@ RUN apt-get --allow-unauthenticated update -y \ unixodbc-dev \ wget \ zlib1g-dev \ - zookeeper \ - zookeeperd RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index b6b48cd0943..adec0f51490 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -48,8 +48,6 @@ fi ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml -service zookeeper start -sleep 5 service clickhouse-server start && sleep 5 if cat /usr/bin/clickhouse-test | grep -q -- "--use-skip-list"; then diff --git a/docker/test/stateless_with_coverage/Dockerfile b/docker/test/stateless_with_coverage/Dockerfile index 77357d5142f..b76989de1cf 100644 --- a/docker/test/stateless_with_coverage/Dockerfile +++ b/docker/test/stateless_with_coverage/Dockerfile @@ -11,8 +11,6 @@ RUN apt-get update -y \ tzdata \ fakeroot \ debhelper \ - zookeeper \ - zookeeperd \ expect \ python \ python-lxml \ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index c3ccb18659b..6ab997925e4 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -72,9 +72,6 @@ ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-clien ln -s --backup=simple --suffix=_original.xml \ /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ -service zookeeper start -sleep 5 - start_clickhouse sleep 10 From bb8a2e161792b31188caf591b1ec7ce51cf267b2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 11:18:36 +0300 Subject: [PATCH 039/131] Better clickhouse-configs --- debian/rules | 4 +-- docker/test/fasttest/run.sh | 30 ++-------------- docker/test/stateful/run.sh | 22 ++---------- docker/test/stateful_with_coverage/run.sh | 24 ++----------- docker/test/stateless/run.sh | 41 ++-------------------- docker/test/stateless_unbundled/Dockerfile | 2 +- docker/test/stateless_unbundled/run.sh | 41 ++-------------------- docker/test/stateless_with_coverage/run.sh | 34 ++---------------- docker/test/stress/run.sh | 5 ++- 9 files changed, 17 insertions(+), 186 deletions(-) diff --git a/debian/rules b/debian/rules index 5b271a8691f..4387053c5aa 100755 --- a/debian/rules +++ b/debian/rules @@ -36,8 +36,8 @@ endif CMAKE_FLAGS += -DENABLE_UTILS=0 -DEB_CC ?= $(shell which gcc-9 gcc-8 gcc | head -n1) -DEB_CXX ?= $(shell which g++-9 g++-8 g++ | head -n1) +DEB_CC ?= $(shell which gcc-10 gcc-9 gcc-8 gcc | head -n1) +DEB_CXX ?= $(shell which g++-10 g++-9 g++-8 g++ | head -n1) ifdef DEB_CXX DEB_BUILD_GNU_TYPE := $(shell dpkg-architecture -qDEB_BUILD_GNU_TYPE) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index ca691e5303c..165bee80b0c 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -111,35 +111,9 @@ ln -s /test_output /var/log/clickhouse-server cp "$CLICKHOUSE_DIR/programs/server/config.xml" /etc/clickhouse-server/ cp "$CLICKHOUSE_DIR/programs/server/users.xml" /etc/clickhouse-server/ -mkdir -p /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ -#ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ -ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml +# install tests config +$CLICKHOUSE_DIR/tests/config/install.sh -# Keep original query_masking_rules.xml -ln -s --backup=simple --suffix=_original.xml /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ # Kill the server in case we are running locally and not in docker kill_clickhouse diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index c3576acc0e4..87cc4054ee6 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -8,26 +8,8 @@ dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb dpkg -i package_folder/clickhouse-test_*.deb -mkdir -p /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ - -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ -fi +# install test configs +/usr/share/clickhouse-test/config/install.sh function start() { diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index c2434b319b9..7191745ec83 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -48,28 +48,8 @@ mkdir -p /var/lib/clickhouse mkdir -p /var/log/clickhouse-server chmod 777 -R /var/log/clickhouse-server/ -# Temorary way to keep CI green while moving dictionaries to separate directory -mkdir -p /etc/clickhouse-server/dict_examples -chmod 777 -R /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/; - -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ - -# Retain any pre-existing config and allow ClickHouse to load those if required -ln -s --backup=simple --suffix=_original.xml \ - /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ +# install test configs +/usr/share/clickhouse-test/config/install.sh function start() { diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index adec0f51490..9f2bb9bf62d 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -8,45 +8,8 @@ dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb dpkg -i package_folder/clickhouse-test_*.deb -mkdir -p /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ - -# Retain any pre-existing config and allow ClickHouse to load it if required -ln -s --backup=simple --suffix=_original.xml \ - /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ - -if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ -fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ -fi - -ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml +# install test configs +/usr/share/clickhouse-test/config/install.sh service clickhouse-server start && sleep 5 diff --git a/docker/test/stateless_unbundled/Dockerfile b/docker/test/stateless_unbundled/Dockerfile index dbb7ae2477f..cb8cd158e5f 100644 --- a/docker/test/stateless_unbundled/Dockerfile +++ b/docker/test/stateless_unbundled/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get --allow-unauthenticated update -y \ unixodbc \ unixodbc-dev \ wget \ - zlib1g-dev \ + zlib1g-dev RUN mkdir -p /tmp/clickhouse-odbc-tmp \ && wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \ diff --git a/docker/test/stateless_unbundled/run.sh b/docker/test/stateless_unbundled/run.sh index adec0f51490..9f2bb9bf62d 100755 --- a/docker/test/stateless_unbundled/run.sh +++ b/docker/test/stateless_unbundled/run.sh @@ -8,45 +8,8 @@ dpkg -i package_folder/clickhouse-server_*.deb dpkg -i package_folder/clickhouse-client_*.deb dpkg -i package_folder/clickhouse-test_*.deb -mkdir -p /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/ -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/custom_settings_prefixes.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ - -# Retain any pre-existing config and allow ClickHouse to load it if required -ln -s --backup=simple --suffix=_original.xml \ - /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ - -if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/polymorphic_parts.xml /etc/clickhouse-server/config.d/ -fi -if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s /usr/share/clickhouse-test/config/database_atomic_configd.xml /etc/clickhouse-server/config.d/ - ln -s /usr/share/clickhouse-test/config/database_atomic_usersd.xml /etc/clickhouse-server/users.d/ -fi - -ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml +# install test configs +/usr/share/clickhouse-test/config/install.sh service clickhouse-server start && sleep 5 diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index 6ab997925e4..2f3f05a335a 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -39,38 +39,8 @@ mkdir -p /var/log/clickhouse-server chmod 777 -R /var/lib/clickhouse chmod 777 -R /var/log/clickhouse-server/ -# Temorary way to keep CI green while moving dictionaries to separate directory -mkdir -p /etc/clickhouse-server/dict_examples -chmod 777 -R /etc/clickhouse-server/dict_examples -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/dict_examples/; \ - ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/dict_examples/; - -ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/access_management.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/executable_dictionary.xml /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/secure_ports.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/clusters.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/graphite.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/server.key /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/server.crt /etc/clickhouse-server/ -ln -s /usr/share/clickhouse-test/config/dhparam.pem /etc/clickhouse-server/ -ln -sf /usr/share/clickhouse-test/config/client_config.xml /etc/clickhouse-client/config.xml - -# Retain any pre-existing config and allow ClickHouse to load it if required -ln -s --backup=simple --suffix=_original.xml \ - /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/ +# install test configs +/usr/share/clickhouse-test/config/install.sh start_clickhouse diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 8295e90b3ef..28c66a72d39 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -39,9 +39,8 @@ function start() done } -ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/ -ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/ -ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/ +# install test configs +/usr/share/clickhouse-test/config/install.sh echo "ASAN_OPTIONS='malloc_context_size=10 verbosity=1 allocator_release_to_os_interval_ms=10000'" >> /etc/environment From 70e6b4093f895e6f8250d5492bc5b597876daf2b Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 24 Sep 2020 11:34:20 +0300 Subject: [PATCH 040/131] Update arcadia_skip_list.txt --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 69391ca9fd4..6d1c6444d1b 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -145,3 +145,4 @@ 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read 01505_distributed_local_type_conversion_enum +01505_log_distributed_deadlock From 0936a3c1f893f6981f15a6f87d5daec4f61ca319 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 11:35:16 +0300 Subject: [PATCH 041/131] Install script for CI --- debian/rules | 4 ++-- release | 10 ++++---- tests/config/install.sh | 52 +++++++++++++++++++++++++++++++++++++++++ 3 files changed, 59 insertions(+), 7 deletions(-) create mode 100755 tests/config/install.sh diff --git a/debian/rules b/debian/rules index 4387053c5aa..837f81dd503 100755 --- a/debian/rules +++ b/debian/rules @@ -36,8 +36,8 @@ endif CMAKE_FLAGS += -DENABLE_UTILS=0 -DEB_CC ?= $(shell which gcc-10 gcc-9 gcc-8 gcc | head -n1) -DEB_CXX ?= $(shell which g++-10 g++-9 g++-8 g++ | head -n1) +DEB_CC ?= $(shell which gcc-10 gcc-9 gcc | head -n1) +DEB_CXX ?= $(shell which g++-10 g++-9 g++ | head -n1) ifdef DEB_CXX DEB_BUILD_GNU_TYPE := $(shell dpkg-architecture -qDEB_BUILD_GNU_TYPE) diff --git a/release b/release index b20683a9caa..b446ceca0d5 100755 --- a/release +++ b/release @@ -95,9 +95,9 @@ then exit 3 fi - export DEB_CC=${DEB_CC=clang-6.0} - export DEB_CXX=${DEB_CXX=clang++-6.0} - EXTRAPACKAGES="$EXTRAPACKAGES clang-6.0 lld-6.0" + export DEB_CC=${DEB_CC=clang-10} + export DEB_CXX=${DEB_CXX=clang++-10} + EXTRAPACKAGES="$EXTRAPACKAGES clang-10 lld-10" elif [[ $BUILD_TYPE == 'valgrind' ]]; then MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0" VERSION_POSTFIX+="+valgrind" @@ -118,8 +118,8 @@ echo -e "\nCurrent version is $VERSION_STRING" if [ -z "$NO_BUILD" ] ; then gen_changelog "$VERSION_STRING" "" "$AUTHOR" "" if [ -z "$USE_PBUILDER" ] ; then - DEB_CC=${DEB_CC:=`which gcc-9 gcc-8 gcc | head -n1`} - DEB_CXX=${DEB_CXX:=`which g++-9 g++-8 g++ | head -n1`} + DEB_CC=${DEB_CC:=`which gcc-10 gcc-9 gcc | head -n1`} + DEB_CXX=${DEB_CXX:=`which gcc-10 g++-9 g++ | head -n1`} # Build (only binary packages). debuild --preserve-env -e PATH \ -e DEB_CC=$DEB_CC -e DEB_CXX=$DEB_CXX -e CMAKE_FLAGS="$CMAKE_FLAGS" \ diff --git a/tests/config/install.sh b/tests/config/install.sh new file mode 100755 index 00000000000..7a005cab5ce --- /dev/null +++ b/tests/config/install.sh @@ -0,0 +1,52 @@ +#!/bin/bash + +# script allows to install configs for clickhouse server and clients required +# for testing (stateless and stateful tests) + +set -x -e + +DEST_SERVER_PATH="${1:-/etc/clickhouse-server}" +DEST_CLIENT_PATH="${2:-/etc/clickhouse-client}" +SRC_PATH="$( cd "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )" + +echo "Going to install test configs from $SRC_PATH into DEST_SERVER_PATH" + +mkdir -p $DEST_SERVER_PATH/config.d/ +mkdir -p $DEST_SERVER_PATH/users.d/ +mkdir -p $DEST_CLIENT_PATH + +ln -s $SRC_PATH/zookeeper.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/listen.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/part_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/text_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/metric_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/log_queries.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/readonly.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/access_management.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/ints_dictionary.xml $DEST_SERVER_PATH/ +ln -s $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ +ln -s $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ +ln -s $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ +ln -s $SRC_PATH/macros.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/disks.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/secure_ports.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/clusters.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/graphite.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/server.key $DEST_SERVER_PATH/ +ln -s $SRC_PATH/server.crt $DEST_SERVER_PATH/ +ln -s $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ + +# Retain any pre-existing config and allow ClickHouse to load it if required +ln -s --backup=simple --suffix=_original.xml \ + $SRC_PATH/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ + +if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then + ln -s $SRC_PATH/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ +fi +if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then + ln -s $SRC_PATH/database_atomic_configd.xml $DEST_SERVER_PATH/config.d/ + ln -s $SRC_PATH/database_atomic_usersd.xml $DEST_SERVER_PATH/users.d/ +fi + +ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From 8cbfd69adf413364e7c0264480ed692c73eb53d5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 24 Sep 2020 11:35:35 +0300 Subject: [PATCH 042/131] Update decimal_casts.xml --- tests/performance/decimal_casts.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/decimal_casts.xml b/tests/performance/decimal_casts.xml index 7f609c8bd9d..582672fa30e 100644 --- a/tests/performance/decimal_casts.xml +++ b/tests/performance/decimal_casts.xml @@ -1,4 +1,4 @@ - + 15G From 4f3eb4532ace4f133b1a680e424e88e63438cb1a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 11:37:20 +0300 Subject: [PATCH 043/131] Add missed changes --- docker/test/fasttest/run.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 165bee80b0c..cf8dbc66870 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -113,6 +113,8 @@ cp "$CLICKHOUSE_DIR/programs/server/users.xml" /etc/clickhouse-server/ # install tests config $CLICKHOUSE_DIR/tests/config/install.sh +# doesn't support SSL +rm -f /etc/clickhouse-server/config.d/secure_ports.xml # Kill the server in case we are running locally and not in docker From a2845acf37b961c8d4b990cc900e1fba7ebec8be Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 24 Sep 2020 16:38:57 +0800 Subject: [PATCH 044/131] Fix joinGet crash with LowCardinality types. --- src/Functions/FunctionJoinGet.h | 2 +- .../0_stateless/01400_join_get_with_multi_keys.reference | 1 + .../queries/0_stateless/01400_join_get_with_multi_keys.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index 6b3b1202f60..e1afd2715f0 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -80,7 +80,7 @@ public: DataTypePtr getReturnType(const ColumnsWithTypeAndName &) const override { return {}; } // Not used bool useDefaultImplementationForNulls() const override { return false; } - bool useDefaultImplementationForLowCardinalityColumns() const override { return true; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } bool isVariadic() const override { return true; } size_t getNumberOfArguments() const override { return 0; } diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference index 49d59571fbf..726b0a9a7a5 100644 --- a/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.reference @@ -1 +1,2 @@ 0.1 +0.1 diff --git a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql index 73068270762..8a19865359b 100644 --- a/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql +++ b/tests/queries/0_stateless/01400_join_get_with_multi_keys.sql @@ -6,4 +6,10 @@ INSERT INTO test_joinGet VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', SELECT joinGet(test_joinGet, 'c', 'ab', '1'); +CREATE TABLE test_lc(a LowCardinality(String), b LowCardinality(String), c Float64) ENGINE = Join(any, left, a, b); + +INSERT INTO test_lc VALUES ('ab', '1', 0.1), ('ab', '2', 0.2), ('cd', '3', 0.3); + +SELECT joinGet(test_lc, 'c', 'ab', '1'); + DROP TABLE test_joinGet; From 8adb9c922482573716c15ca4f063bc5eed7d9676 Mon Sep 17 00:00:00 2001 From: Artem Zuikov Date: Thu, 24 Sep 2020 12:39:55 +0300 Subject: [PATCH 045/131] test for fixed issue (#15189) --- .../01482_move_to_prewhere_and_cast.reference | 1 + .../01482_move_to_prewhere_and_cast.sql | 31 +++++++++++++++++++ 2 files changed, 32 insertions(+) create mode 100644 tests/queries/0_stateless/01482_move_to_prewhere_and_cast.reference create mode 100644 tests/queries/0_stateless/01482_move_to_prewhere_and_cast.sql diff --git a/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.reference b/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.reference new file mode 100644 index 00000000000..29597554bbc --- /dev/null +++ b/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.reference @@ -0,0 +1 @@ +ApplicationA 2020-01-01 diff --git a/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.sql b/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.sql new file mode 100644 index 00000000000..b79a3cf05b4 --- /dev/null +++ b/tests/queries/0_stateless/01482_move_to_prewhere_and_cast.sql @@ -0,0 +1,31 @@ +DROP TABLE IF EXISTS APPLICATION; +DROP TABLE IF EXISTS DATABASE_IO; + +CREATE TABLE APPLICATION ( + `Name` LowCardinality(String), + `Base` LowCardinality(String) +) ENGINE = Memory(); + +insert into table APPLICATION values ('ApplicationA', 'BaseA'), ('ApplicationB', 'BaseB') , ('ApplicationC', 'BaseC'); + +CREATE TABLE DATABASE_IO ( + `Application` LowCardinality(String), + `Base` LowCardinality(String), + `Date` DateTime, + `Ios` UInt32 ) +ENGINE = MergeTree() +ORDER BY Date; + +insert into table DATABASE_IO values ('AppA', 'BaseA', '2020-01-01 00:00:00', 1000); + +SELECT `APPLICATION`.`Name` AS `App`, + CAST(CAST(`DATABASE_IO`.`Date` AS DATE) AS DATE) AS `date` +FROM `DATABASE_IO` +INNER +JOIN `APPLICATION` ON (`DATABASE_IO`.`Base` = `APPLICATION`.`Base`) +WHERE ( + CAST(CAST(`DATABASE_IO`.`Date` AS DATE) AS TIMESTAMP) >= toDateTime('2020-01-01 00:00:00') +); + +DROP TABLE APPLICATION; +DROP TABLE DATABASE_IO; From b8861ad6cc00cd60330f253dac999053dde8ce92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:21:34 +0300 Subject: [PATCH 046/131] Move config files to directories --- tests/config/{ => config.d}/clusters.xml | 0 tests/config/{ => config.d}/custom_settings_prefixes.xml | 0 tests/config/{ => config.d}/database_atomic_configd.xml | 0 tests/config/{ => config.d}/disks.xml | 0 tests/config/{ => config.d}/graphite.xml | 0 tests/config/{ => config.d}/listen.xml | 0 tests/config/{ => config.d}/macros.xml | 0 tests/config/{ => config.d}/metric_log.xml | 0 tests/config/{ => config.d}/part_log.xml | 0 tests/config/{ => config.d}/polymorphic_parts.xml | 0 tests/config/{ => config.d}/query_masking_rules.xml | 0 tests/config/{ => config.d}/secure_ports.xml | 0 tests/config/{ => config.d}/text_log.xml | 0 tests/config/{ => config.d}/zookeeper.xml | 0 tests/config/{ => users.d}/access_management.xml | 0 tests/config/{ => users.d}/database_atomic_usersd.xml | 0 tests/config/{ => users.d}/log_queries.xml | 0 tests/config/{ => users.d}/readonly.xml | 0 18 files changed, 0 insertions(+), 0 deletions(-) rename tests/config/{ => config.d}/clusters.xml (100%) rename tests/config/{ => config.d}/custom_settings_prefixes.xml (100%) rename tests/config/{ => config.d}/database_atomic_configd.xml (100%) rename tests/config/{ => config.d}/disks.xml (100%) rename tests/config/{ => config.d}/graphite.xml (100%) rename tests/config/{ => config.d}/listen.xml (100%) rename tests/config/{ => config.d}/macros.xml (100%) rename tests/config/{ => config.d}/metric_log.xml (100%) rename tests/config/{ => config.d}/part_log.xml (100%) rename tests/config/{ => config.d}/polymorphic_parts.xml (100%) rename tests/config/{ => config.d}/query_masking_rules.xml (100%) rename tests/config/{ => config.d}/secure_ports.xml (100%) rename tests/config/{ => config.d}/text_log.xml (100%) rename tests/config/{ => config.d}/zookeeper.xml (100%) rename tests/config/{ => users.d}/access_management.xml (100%) rename tests/config/{ => users.d}/database_atomic_usersd.xml (100%) rename tests/config/{ => users.d}/log_queries.xml (100%) rename tests/config/{ => users.d}/readonly.xml (100%) diff --git a/tests/config/clusters.xml b/tests/config/config.d/clusters.xml similarity index 100% rename from tests/config/clusters.xml rename to tests/config/config.d/clusters.xml diff --git a/tests/config/custom_settings_prefixes.xml b/tests/config/config.d/custom_settings_prefixes.xml similarity index 100% rename from tests/config/custom_settings_prefixes.xml rename to tests/config/config.d/custom_settings_prefixes.xml diff --git a/tests/config/database_atomic_configd.xml b/tests/config/config.d/database_atomic_configd.xml similarity index 100% rename from tests/config/database_atomic_configd.xml rename to tests/config/config.d/database_atomic_configd.xml diff --git a/tests/config/disks.xml b/tests/config/config.d/disks.xml similarity index 100% rename from tests/config/disks.xml rename to tests/config/config.d/disks.xml diff --git a/tests/config/graphite.xml b/tests/config/config.d/graphite.xml similarity index 100% rename from tests/config/graphite.xml rename to tests/config/config.d/graphite.xml diff --git a/tests/config/listen.xml b/tests/config/config.d/listen.xml similarity index 100% rename from tests/config/listen.xml rename to tests/config/config.d/listen.xml diff --git a/tests/config/macros.xml b/tests/config/config.d/macros.xml similarity index 100% rename from tests/config/macros.xml rename to tests/config/config.d/macros.xml diff --git a/tests/config/metric_log.xml b/tests/config/config.d/metric_log.xml similarity index 100% rename from tests/config/metric_log.xml rename to tests/config/config.d/metric_log.xml diff --git a/tests/config/part_log.xml b/tests/config/config.d/part_log.xml similarity index 100% rename from tests/config/part_log.xml rename to tests/config/config.d/part_log.xml diff --git a/tests/config/polymorphic_parts.xml b/tests/config/config.d/polymorphic_parts.xml similarity index 100% rename from tests/config/polymorphic_parts.xml rename to tests/config/config.d/polymorphic_parts.xml diff --git a/tests/config/query_masking_rules.xml b/tests/config/config.d/query_masking_rules.xml similarity index 100% rename from tests/config/query_masking_rules.xml rename to tests/config/config.d/query_masking_rules.xml diff --git a/tests/config/secure_ports.xml b/tests/config/config.d/secure_ports.xml similarity index 100% rename from tests/config/secure_ports.xml rename to tests/config/config.d/secure_ports.xml diff --git a/tests/config/text_log.xml b/tests/config/config.d/text_log.xml similarity index 100% rename from tests/config/text_log.xml rename to tests/config/config.d/text_log.xml diff --git a/tests/config/zookeeper.xml b/tests/config/config.d/zookeeper.xml similarity index 100% rename from tests/config/zookeeper.xml rename to tests/config/config.d/zookeeper.xml diff --git a/tests/config/access_management.xml b/tests/config/users.d/access_management.xml similarity index 100% rename from tests/config/access_management.xml rename to tests/config/users.d/access_management.xml diff --git a/tests/config/database_atomic_usersd.xml b/tests/config/users.d/database_atomic_usersd.xml similarity index 100% rename from tests/config/database_atomic_usersd.xml rename to tests/config/users.d/database_atomic_usersd.xml diff --git a/tests/config/log_queries.xml b/tests/config/users.d/log_queries.xml similarity index 100% rename from tests/config/log_queries.xml rename to tests/config/users.d/log_queries.xml diff --git a/tests/config/readonly.xml b/tests/config/users.d/readonly.xml similarity index 100% rename from tests/config/readonly.xml rename to tests/config/users.d/readonly.xml From 2846645fa3482aae0b6469f810ec531e32d42fdf Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:22:00 +0300 Subject: [PATCH 047/131] Update install.sh --- tests/config/install.sh | 40 +++++++++++++++++++++------------------- 1 file changed, 21 insertions(+), 19 deletions(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 7a005cab5ce..20641833815 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -9,44 +9,46 @@ DEST_SERVER_PATH="${1:-/etc/clickhouse-server}" DEST_CLIENT_PATH="${2:-/etc/clickhouse-client}" SRC_PATH="$( cd "$(dirname "$0")" >/dev/null 2>&1 ; pwd -P )" -echo "Going to install test configs from $SRC_PATH into DEST_SERVER_PATH" +echo "Going to install test configs from $SRC_PATH into $DEST_SERVER_PATH" mkdir -p $DEST_SERVER_PATH/config.d/ mkdir -p $DEST_SERVER_PATH/users.d/ mkdir -p $DEST_CLIENT_PATH -ln -s $SRC_PATH/zookeeper.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/listen.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/part_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/text_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/metric_log.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/log_queries.xml $DEST_SERVER_PATH/users.d/ -ln -s $SRC_PATH/readonly.xml $DEST_SERVER_PATH/users.d/ -ln -s $SRC_PATH/access_management.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/config.d/zookeeper.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/listen.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/part_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/text_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/metric_log.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/custom_settings_prefixes.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/macros.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/disks.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/secure_ports.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/clusters.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/config.d/graphite.xml $DEST_SERVER_PATH/config.d/ +ln -s $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/users.d/readonly.xml $DEST_SERVER_PATH/users.d/ +ln -s $SRC_PATH/users.d/access_management.xml $DEST_SERVER_PATH/users.d/ + ln -s $SRC_PATH/ints_dictionary.xml $DEST_SERVER_PATH/ ln -s $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/ ln -s $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/ ln -s $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/ -ln -s $SRC_PATH/macros.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/disks.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/secure_ports.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/clusters.xml $DEST_SERVER_PATH/config.d/ -ln -s $SRC_PATH/graphite.xml $DEST_SERVER_PATH/config.d/ + ln -s $SRC_PATH/server.key $DEST_SERVER_PATH/ ln -s $SRC_PATH/server.crt $DEST_SERVER_PATH/ ln -s $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ # Retain any pre-existing config and allow ClickHouse to load it if required ln -s --backup=simple --suffix=_original.xml \ - $SRC_PATH/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ + $SRC_PATH/config.d/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then - ln -s $SRC_PATH/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ + ln -s $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ fi if [[ -n "$USE_DATABASE_ATOMIC" ]] && [[ "$USE_DATABASE_ATOMIC" -eq 1 ]]; then - ln -s $SRC_PATH/database_atomic_configd.xml $DEST_SERVER_PATH/config.d/ - ln -s $SRC_PATH/database_atomic_usersd.xml $DEST_SERVER_PATH/users.d/ + ln -s $SRC_PATH/config.d/database_atomic_configd.xml $DEST_SERVER_PATH/config.d/ + ln -s $SRC_PATH/users.d/database_atomic_usersd.xml $DEST_SERVER_PATH/users.d/ fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From 24cb88f808471a65ce4c4a183a5284bcd0177177 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:28:13 +0300 Subject: [PATCH 048/131] Add readme --- tests/config/README.md | 8 ++++++++ 1 file changed, 8 insertions(+) create mode 100644 tests/config/README.md diff --git a/tests/config/README.md b/tests/config/README.md new file mode 100644 index 00000000000..b172c8afea3 --- /dev/null +++ b/tests/config/README.md @@ -0,0 +1,8 @@ +# ClickHouse configs for test environment + +## How to use +If you want to run all tests from `tests/queries/0_stateless` and `test/queries/0_stateful` on your local machine you have to set up configs from this directory for your clickhouse-server. This most simple way is to install them using `install.sh` script. + +## How to add new config + +Just place file `.xml` with new config into appropriate directory and add `ln` command into `install.sh` script. After that CI will use this config in all tests runs. From 7277cbe1e69e60d8fbe1af225ecc10de2f5905b9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:28:56 +0300 Subject: [PATCH 049/131] Fix readme --- tests/config/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/README.md b/tests/config/README.md index b172c8afea3..7e082403913 100644 --- a/tests/config/README.md +++ b/tests/config/README.md @@ -1,7 +1,7 @@ # ClickHouse configs for test environment ## How to use -If you want to run all tests from `tests/queries/0_stateless` and `test/queries/0_stateful` on your local machine you have to set up configs from this directory for your clickhouse-server. This most simple way is to install them using `install.sh` script. +If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your clickhouse-server. This most simple way is to install them using `install.sh` script. ## How to add new config From 912d45c443220d238556059650498665e72d944f Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:31:21 +0300 Subject: [PATCH 050/131] Fix readme --- tests/config/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/README.md b/tests/config/README.md index 7e082403913..faa0e922f13 100644 --- a/tests/config/README.md +++ b/tests/config/README.md @@ -1,7 +1,7 @@ # ClickHouse configs for test environment ## How to use -If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your clickhouse-server. This most simple way is to install them using `install.sh` script. +CI use this configs in all checks installing them with `install.sh` script. If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your `clickhouse-server`. The most simple way is to install them using `install.sh` script. Other option is just copy files into your clickhouse config directory. ## How to add new config From f8ab9896e7142348d9c60087c77b4c16fcfd42fb Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:32:49 +0300 Subject: [PATCH 051/131] One more time --- tests/config/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/README.md b/tests/config/README.md index faa0e922f13..8dd775a275a 100644 --- a/tests/config/README.md +++ b/tests/config/README.md @@ -1,7 +1,7 @@ # ClickHouse configs for test environment ## How to use -CI use this configs in all checks installing them with `install.sh` script. If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your `clickhouse-server`. The most simple way is to install them using `install.sh` script. Other option is just copy files into your clickhouse config directory. +CI use these configs in all checks installing them with `install.sh` script. If you want to run all tests from `tests/queries/0_stateless` and `test/queries/1_stateful` on your local machine you have to set up configs from this directory for your `clickhouse-server`. The most simple way is to install them using `install.sh` script. Other option is just copy files into your clickhouse config directory. ## How to add new config From e65f1098e27a664e83ce319ae6938573e8df547b Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 24 Sep 2020 13:44:13 +0300 Subject: [PATCH 052/131] Bump CI. From e7a764c2c5a6d2334ff80a8362dea549d8117582 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Sep 2020 13:49:36 +0300 Subject: [PATCH 053/131] fix flacky test --- .../0_stateless/01320_create_sync_race_condition_zookeeper.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index f72c9da27ba..cc6a66bd6bc 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -26,4 +26,4 @@ timeout 10 bash -c thread2 & wait -$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" 2>&1 | grep -v "New table appeared in database being dropped or detached. Try again." || exit 0 +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01320" 2>&1 | grep -F "Code:" | grep -v "New table appeared in database being dropped or detached" || exit 0 From 41ad365eafaee7cdba978967414467dc9b8b18f4 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 24 Sep 2020 14:11:59 +0300 Subject: [PATCH 054/131] Increase fast test threads to 8 Tests don't often fail in parallel (1/10 runs), so we can make them faster. --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index ca691e5303c..4efacc7cb9e 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -216,7 +216,7 @@ TESTS_TO_SKIP=( 01460_DistributedFilesToInsert ) -clickhouse-test -j 4 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt +clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt # substr is to remove semicolon after test name From 5f9ef1ebbf5acf0d3b616abfd1f68ba12704b0fd Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 24 Sep 2020 14:50:18 +0300 Subject: [PATCH 055/131] Update README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f1c8e17086b..6b909dd710c 100644 --- a/README.md +++ b/README.md @@ -17,5 +17,5 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events -* [eBay migrating from Druid](https://us02web.zoom.us/webinar/register/tZMkfu6rpjItHtaQ1DXcgPWcSOnmM73HLGKL) on September 23, 2020. * [ClickHouse for Edge Analytics](https://ones2020.sched.com/event/bWPs) on September 29, 2020. +* [ClickHouse online meetup (in Russian)](https://clck.ru/R2zB9) on October 1, 2020. From cc305cf88667009943dbfc43c2bcd42b8cbc9755 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 15:34:03 +0300 Subject: [PATCH 056/131] Fix fuzzer image --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index bcac5a433cc..d35a13cc421 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -48,7 +48,7 @@ function configure cp -av "$repo_dir"/programs/server/config* db cp -av "$repo_dir"/programs/server/user* db # TODO figure out which ones are needed - cp -av "$repo_dir"/tests/config/listen.xml db/config.d + cp -av "$repo_dir"/tests/config/config.d/listen.xml db/config.d cp -av "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d } From c298c633a793e52543f38df78ef0e8098be6f0d6 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 16:10:24 +0300 Subject: [PATCH 057/131] Revert "Test and doc for PR12771 krb5 + cyrus-sasl + kerberized kafka" --- .gitmodules | 1 - cmake/find/rdkafka.cmake | 4 +- contrib/cyrus-sasl | 2 +- docker/images.json | 4 - docker/test/integration/base/Dockerfile | 3 +- .../test/integration/kerberos_kdc/Dockerfile | 15 -- .../docker_compose_kerberized_kafka.yml | 59 ------- .../integration/runner/dockerd-entrypoint.sh | 1 - .../table-engines/integrations/kafka.md | 16 -- tests/integration/helpers/cluster.py | 40 +---- tests/integration/runner | 2 - .../test_storage_kerberized_kafka/__init__.py | 0 .../clickhouse_path/EMPTY_DIR | 0 .../configs/kafka.xml | 26 ---- .../configs/log_conf.xml | 11 -- .../kerberos_image_config.sh | 132 ---------------- .../secrets/broker_jaas.conf | 14 -- .../secrets/krb.conf | 22 --- .../secrets/zookeeper_jaas.conf | 14 -- .../test_storage_kerberized_kafka/test.py | 146 ------------------ 20 files changed, 7 insertions(+), 505 deletions(-) delete mode 100644 docker/test/integration/kerberos_kdc/Dockerfile delete mode 100644 docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml delete mode 100644 tests/integration/test_storage_kerberized_kafka/__init__.py delete mode 100644 tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR delete mode 100644 tests/integration/test_storage_kerberized_kafka/configs/kafka.xml delete mode 100644 tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml delete mode 100644 tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh delete mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf delete mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/krb.conf delete mode 100644 tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf delete mode 100644 tests/integration/test_storage_kerberized_kafka/test.py diff --git a/.gitmodules b/.gitmodules index 865a876b276..eb21c4bfd00 100644 --- a/.gitmodules +++ b/.gitmodules @@ -186,4 +186,3 @@ [submodule "contrib/cyrus-sasl"] path = contrib/cyrus-sasl url = https://github.com/cyrusimap/cyrus-sasl - branch = cyrus-sasl-2.1 diff --git a/cmake/find/rdkafka.cmake b/cmake/find/rdkafka.cmake index ac11322f408..d9f815dbcdd 100644 --- a/cmake/find/rdkafka.cmake +++ b/cmake/find/rdkafka.cmake @@ -14,10 +14,10 @@ if (NOT ENABLE_RDKAFKA) return() endif() -if (NOT ARCH_ARM) +if (NOT ARCH_ARM AND USE_LIBGSASL) option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) elseif(USE_INTERNAL_RDKAFKA_LIBRARY) - message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM}") + message (${RECONFIGURE_MESSAGE_LEVEL} "Can't use internal librdkafka with ARCH_ARM=${ARCH_ARM} AND USE_LIBGSASL=${USE_LIBGSASL}") endif () if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/cppkafka/CMakeLists.txt") diff --git a/contrib/cyrus-sasl b/contrib/cyrus-sasl index 9995bf9d8e1..6054630889f 160000 --- a/contrib/cyrus-sasl +++ b/contrib/cyrus-sasl @@ -1 +1 @@ -Subproject commit 9995bf9d8e14f58934d9313ac64f13780d6dd3c9 +Subproject commit 6054630889fd1cd8d0659573d69badcee1e23a00 diff --git a/docker/images.json b/docker/images.json index e9e91864e1e..8c2cb35b004 100644 --- a/docker/images.json +++ b/docker/images.json @@ -133,10 +133,6 @@ "name": "yandex/clickhouse-postgresql-java-client", "dependent": [] }, - "docker/test/integration/kerberos_kdc": { - "name": "yandex/clickhouse-kerberos-kdc", - "dependent": [] - }, "docker/test/base": { "name": "yandex/clickhouse-test-base", "dependent": [ diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 3e4e88965e0..35decd907c0 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -16,8 +16,7 @@ RUN apt-get update \ odbc-postgresql \ sqlite3 \ curl \ - tar \ - krb5-user + tar RUN rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/kerberos_kdc/Dockerfile b/docker/test/integration/kerberos_kdc/Dockerfile deleted file mode 100644 index ea231b1191d..00000000000 --- a/docker/test/integration/kerberos_kdc/Dockerfile +++ /dev/null @@ -1,15 +0,0 @@ -# docker build -t yandex/clickhouse-kerberos-kdc . - -FROM centos:6.6 -# old OS to make is faster and smaller - -RUN yum install -y krb5-server krb5-libs krb5-auth-dialog krb5-workstation - -EXPOSE 88 749 - -RUN touch /config.sh -# should be overwritten e.g. via docker_compose volumes -# volumes: /some_path/my_kerberos_config.sh:/config.sh:ro - - -ENTRYPOINT ["/bin/bash", "/config.sh"] diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml deleted file mode 100644 index 3ce0000b148..00000000000 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ /dev/null @@ -1,59 +0,0 @@ -version: '2.3' - -services: - kafka_kerberized_zookeeper: - image: confluentinc/cp-zookeeper:5.2.0 - # restart: always - hostname: kafka_kerberized_zookeeper - environment: - ZOOKEEPER_SERVER_ID: 1 - ZOOKEEPER_CLIENT_PORT: 2181 - ZOOKEEPER_SERVERS: "kafka_kerberized_zookeeper:2888:3888" - KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/zookeeper_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dzookeeper.authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider -Dsun.security.krb5.debug=true" - volumes: - - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets - - /dev/urandom:/dev/random - depends_on: - - kafka_kerberos - security_opt: - - label:disable - - kerberized_kafka1: - image: confluentinc/cp-kafka:5.2.0 - # restart: always - hostname: kerberized_kafka1 - ports: - - "9092:9092" - - "9093:9093" - environment: - KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 - KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 - # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 - # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 - KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI - KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI - KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka - KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: OUTSIDE:SASL_PLAINTEXT,UNSECURED_OUTSIDE:PLAINTEXT,UNSECURED_INSIDE:PLAINTEXT, - KAFKA_INTER_BROKER_LISTENER_NAME: OUTSIDE - KAFKA_BROKER_ID: 1 - KAFKA_ZOOKEEPER_CONNECT: "kafka_kerberized_zookeeper:2181" - KAFKA_LOG4J_LOGGERS: "kafka.controller=INFO,kafka.producer.async.DefaultEventHandler=INFO,state.change.logger=INFO" - KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 - KAFKA_OPTS: "-Djava.security.auth.login.config=/etc/kafka/secrets/broker_jaas.conf -Djava.security.krb5.conf=/etc/kafka/secrets/krb.conf -Dsun.security.krb5.debug=true" - volumes: - - ${KERBERIZED_KAFKA_DIR}/secrets:/etc/kafka/secrets - - /dev/urandom:/dev/random - depends_on: - - kafka_kerberized_zookeeper - - kafka_kerberos - security_opt: - - label:disable - - kafka_kerberos: - image: yandex/clickhouse-kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG} - hostname: kafka_kerberos - volumes: - - ${KERBERIZED_KAFKA_DIR}/secrets:/tmp/keytab - - ${KERBERIZED_KAFKA_DIR}/../../kerberos_image_config.sh:/config.sh - - /dev/urandom:/dev/random - ports: [88, 749] diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index cbdb7317b1e..c38260279ed 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -27,7 +27,6 @@ export DOCKER_MYSQL_JAVA_CLIENT_TAG=${DOCKER_MYSQL_JAVA_CLIENT_TAG:=latest} export DOCKER_MYSQL_JS_CLIENT_TAG=${DOCKER_MYSQL_JS_CLIENT_TAG:=latest} export DOCKER_MYSQL_PHP_CLIENT_TAG=${DOCKER_MYSQL_PHP_CLIENT_TAG:=latest} export DOCKER_POSTGRESQL_JAVA_CLIENT_TAG=${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:=latest} -export DOCKER_KERBEROS_KDC_TAG=${DOCKER_KERBEROS_KDC_TAG:=latest} cd /ClickHouse/tests/integration exec "$@" diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index d0a4bc928a7..fe9aa2ca25e 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -165,22 +165,6 @@ Similar to GraphiteMergeTree, the Kafka engine supports extended configuration u For a list of possible configuration options, see the [librdkafka configuration reference](https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md). Use the underscore (`_`) instead of a dot in the ClickHouse configuration. For example, `check.crcs=true` will be `true`. -### Kerberos support {#kafka-kerberos-support} - -To deal with Kerberos-aware Kafka, add `security_protocol` child element with `sasl_plaintext` value. It is enough if Kerberos ticket-granting ticket is obtained and cached by OS facilities. -ClickHouse is able to maintain Kerberos credentials using a keytab file. Consider `sasl_kerberos_service_name`, `sasl_kerberos_keytab`, `sasl_kerberos_principal` and `sasl.kerberos.kinit.cmd` child elements. - -Example: - -``` xml - - - SASL_PLAINTEXT - /home/kafkauser/kafkauser.keytab - kafkauser/kafkahost@EXAMPLE.COM - -``` - ## Virtual Columns {#virtual-columns} - `_topic` — Kafka topic. diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9ae24e3dafd..6d0f038daed 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -45,6 +45,7 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): f.write("=".join([var, value]) + "\n") return full_path + def subprocess_check_call(args): # Uncomment for debugging # print('run:', ' ' . join(args)) @@ -124,7 +125,6 @@ class ClickHouseCluster: self.base_zookeeper_cmd = None self.base_mysql_cmd = [] self.base_kafka_cmd = [] - self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] self.base_cassandra_cmd = [] self.pre_zookeeper_commands = [] @@ -133,7 +133,6 @@ class ClickHouseCluster: self.with_mysql = False self.with_postgres = False self.with_kafka = False - self.with_kerberized_kafka = False self.with_rabbitmq = False self.with_odbc_drivers = False self.with_hdfs = False @@ -170,7 +169,7 @@ class ClickHouseCluster: def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, + with_zookeeper=False, with_mysql=False, with_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -208,7 +207,6 @@ class ClickHouseCluster: zookeeper_config_path=self.zookeeper_config_path, with_mysql=with_mysql, with_kafka=with_kafka, - with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, with_mongo=with_mongo, with_redis=with_redis, @@ -292,13 +290,6 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) - if with_kerberized_kafka and not self.with_kerberized_kafka: - self.with_kerberized_kafka = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_kerberized_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name', - self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] - cmds.append(self.base_kerberized_kafka_cmd) - if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) @@ -617,11 +608,6 @@ class ClickHouseCluster: self.kafka_docker_id = self.get_instance_docker_id('kafka1') self.wait_schema_registry_to_start(120) - if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: - env = os.environ.copy() - env['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) - self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') @@ -802,12 +788,9 @@ services: - {instance_config_dir}:/etc/clickhouse-server/ - {db_dir}:/var/lib/clickhouse/ - {logs_dir}:/var/log/clickhouse-server/ - - /etc/passwd:/etc/passwd:ro {binary_volume} {odbc_bridge_volume} {odbc_ini_path} - {keytab_path} - {krb5_conf} entrypoint: {entrypoint_cmd} tmpfs: {tmpfs} cap_add: @@ -837,7 +820,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_mongo, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_rabbitmq, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -856,7 +839,6 @@ class ClickHouseInstance: self.custom_user_config_paths = [p.abspath(p.join(base_path, c)) for c in custom_user_configs] self.custom_dictionaries_paths = [p.abspath(p.join(base_path, c)) for c in custom_dictionaries] self.clickhouse_path_dir = p.abspath(p.join(base_path, clickhouse_path_dir)) if clickhouse_path_dir else None - self.kerberos_secrets_dir = p.abspath(p.join(base_path, 'secrets')) self.macros = macros if macros is not None else {} self.with_zookeeper = with_zookeeper self.zookeeper_config_path = zookeeper_config_path @@ -866,7 +848,6 @@ class ClickHouseInstance: self.with_mysql = with_mysql self.with_kafka = with_kafka - self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq self.with_mongo = with_mongo self.with_redis = with_redis @@ -882,13 +863,6 @@ class ClickHouseInstance: else: self.odbc_ini_path = "" - if with_kerberized_kafka: - self.keytab_path = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets:/tmp/keytab" - self.krb5_conf = '- ' + os.path.dirname(self.docker_compose_path) + "/secrets/krb.conf:/etc/krb5.conf:ro" - else: - self.keytab_path = "" - self.krb5_conf = "" - self.docker_client = None self.ip_address = None self.client = None @@ -1218,9 +1192,6 @@ class ClickHouseInstance: if self.with_zookeeper: shutil.copy(self.zookeeper_config_path, conf_d_dir) - if self.with_kerberized_kafka: - shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) - # Copy config.d configs print "Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir) for path in self.custom_main_config_paths: @@ -1256,9 +1227,6 @@ class ClickHouseInstance: depends_on.append("kafka1") depends_on.append("schema-registry") - if self.with_kerberized_kafka: - depends_on.append("kerberized_kafka1") - if self.with_rabbitmq: depends_on.append("rabbitmq1") @@ -1322,8 +1290,6 @@ class ClickHouseInstance: user=os.getuid(), env_file=env_file, odbc_ini_path=odbc_ini_path, - keytab_path=self.keytab_path, - krb5_conf=self.krb5_conf, entrypoint_cmd=entrypoint_cmd, networks=networks, app_net=app_net, diff --git a/tests/integration/runner b/tests/integration/runner index f38ab0aa042..f097a42e52a 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -156,8 +156,6 @@ if __name__ == "__main__": env_tags += "-e {}={} ".format("DOCKER_POSTGRESQL_JAVA_CLIENT_TAG", tag) elif image == "yandex/clickhouse-integration-test": env_tags += "-e {}={}".format("DOCKER_BASE_TAG", tag) - elif image == "yandex/clickhouse-kerberos-kdc": - env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: logging.info("Unknown image {}".format(image)) diff --git a/tests/integration/test_storage_kerberized_kafka/__init__.py b/tests/integration/test_storage_kerberized_kafka/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR b/tests/integration/test_storage_kerberized_kafka/clickhouse_path/EMPTY_DIR deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml b/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml deleted file mode 100644 index 0302bd78e3f..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/configs/kafka.xml +++ /dev/null @@ -1,26 +0,0 @@ - - - earliest - - SASL_PLAINTEXT - GSSAPI - kafka - /tmp/keytab/clickhouse.keytab - kafkauser/instance@TEST.CLICKHOUSE.TECH - security - false - - - - - 300 - - 6000 - - diff --git a/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml b/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml deleted file mode 100644 index 95466269afe..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/configs/log_conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - trace - /var/log/clickhouse-server/log.log - /var/log/clickhouse-server/log.err.log - 1000M - 10 - /var/log/clickhouse-server/stderr.log - /var/log/clickhouse-server/stdout.log - - \ No newline at end of file diff --git a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh b/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh deleted file mode 100644 index dda10d47d94..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/kerberos_image_config.sh +++ /dev/null @@ -1,132 +0,0 @@ -#!/bin/bash - - -set -x # trace - -: "${REALM:=TEST.CLICKHOUSE.TECH}" -: "${DOMAIN_REALM:=test.clickhouse.tech}" -: "${KERB_MASTER_KEY:=masterkey}" -: "${KERB_ADMIN_USER:=admin}" -: "${KERB_ADMIN_PASS:=admin}" - -create_config() { - : "${KDC_ADDRESS:=$(hostname -f)}" - - cat>/etc/krb5.conf</var/kerberos/krb5kdc/kdc.conf< /var/kerberos/krb5kdc/kadm5.acl -} - -create_keytabs() { - - kadmin.local -q "addprinc -randkey zookeeper/kafka_kerberized_zookeeper@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kafka_kerberized_zookeeper.keytab zookeeper/kafka_kerberized_zookeeper@${REALM}" - - kadmin.local -q "addprinc -randkey kafka/kerberized_kafka1@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/kerberized_kafka.keytab kafka/kerberized_kafka1@${REALM}" - - kadmin.local -q "addprinc -randkey zkclient@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/zkclient.keytab zkclient@${REALM}" - - - kadmin.local -q "addprinc -randkey kafkauser/instance@${REALM}" - kadmin.local -q "ktadd -norandkey -k /tmp/keytab/clickhouse.keytab kafkauser/instance@${REALM}" - - chmod g+r /tmp/keytab/clickhouse.keytab - -} - -main() { - - if [ ! -f /kerberos_initialized ]; then - create_config - create_db - create_admin_user - start_kdc - - touch /kerberos_initialized - fi - - if [ ! -f /var/kerberos/krb5kdc/principal ]; then - while true; do sleep 1000; done - else - start_kdc - create_keytabs - tail -F /var/log/kerberos/krb5kdc.log - fi - -} - -[[ "$0" == "${BASH_SOURCE[0]}" ]] && main "$@" diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf deleted file mode 100644 index 8a55ec2faa0..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/secrets/broker_jaas.conf +++ /dev/null @@ -1,14 +0,0 @@ -KafkaServer { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - storeKey=true - keyTab="/etc/kafka/secrets/kerberized_kafka.keytab" - principal="kafka/kerberized_kafka1@TEST.CLICKHOUSE.TECH"; -}; -Client { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - storeKey=true - keyTab="/etc/kafka/secrets/zkclient.keytab" - principal="zkclient@TEST.CLICKHOUSE.TECH"; -}; diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf b/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf deleted file mode 100644 index 1efdf510f22..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/secrets/krb.conf +++ /dev/null @@ -1,22 +0,0 @@ -[logging] - default = FILE:/var/log/kerberos/krb5libs.log - kdc = FILE:/var/log/kerberos/krb5kdc.log - admin_server = FILE:/var/log/kerberos/kadmind.log - -[libdefaults] - default_realm = TEST.CLICKHOUSE.TECH - dns_lookup_realm = false - dns_lookup_kdc = false - ticket_lifetime = 15s - renew_lifetime = 15s - forwardable = true - -[realms] - TEST.CLICKHOUSE.TECH = { - kdc = kafka_kerberos - admin_server = kafka_kerberos - } - -[domain_realm] - .TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH - TEST.CLICKHOUSE.TECH = TEST.CLICKHOUSE.TECH diff --git a/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf b/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf deleted file mode 100644 index 1b1f8103f42..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/secrets/zookeeper_jaas.conf +++ /dev/null @@ -1,14 +0,0 @@ -Server { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - storeKey=true - keyTab="/etc/kafka/secrets/kafka_kerberized_zookeeper.keytab" - principal="zookeeper/kafka_kerberized_zookeeper@TEST.CLICKHOUSE.TECH"; -}; -Client { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - storeKey=true - keyTab="/etc/kafka/secrets/zkclient.keytab" - principal="zkclient@TEST.CLICKHOUSE.TECH"; -}; diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py deleted file mode 100644 index ec23d340977..00000000000 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ /dev/null @@ -1,146 +0,0 @@ -import os.path as p -import random -import threading -import time -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV -from helpers.client import QueryRuntimeException -from helpers.network import PartitionManager - -import json -import subprocess -import kafka.errors -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection -from kafka.admin import NewTopic -from kafka.protocol.admin import DescribeGroupsResponse_v1, DescribeGroupsRequest_v1 -from kafka.protocol.group import MemberAssignment -import socket - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ], - with_kerberized_kafka=True, - clickhouse_path_dir="clickhouse_path" - ) -kafka_id = '' # instance.cluster.kafka_docker_id - -# Helpers - -def check_kafka_is_available(): - - # plaintext - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'localhost:9093'), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise "Kafka is not available" - print("Waiting for Kafka to start up") - time.sleep(1) - - -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9093") - for message in messages: - producer.send(topic=topic, value=message, timestamp_ms=timestamp) - producer.flush() - print ("Produced {} messages for topic {}".format(len(messages), topic)) - - - -# Fixtures - -@pytest.fixture(scope="module") -def kafka_cluster(): - try: - global kafka_id - cluster.start() - kafka_id = instance.cluster.kerberized_kafka_docker_id - print("kafka_id is {}".format(kafka_id)) - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(autouse=True) -def kafka_setup_teardown(): - instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() - print("kafka is available - running test") - yield # run test - -# Tests - -@pytest.mark.timeout(180) # wait to build containers -def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) - - instance.query(''' - CREATE TABLE test.kafka (field String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', - kafka_topic_list = 'kafka_json_as_string', - kafka_group_name = 'kafka_json_as_string', - kafka_format = 'JSONAsString', - kafka_flush_interval_ms=1000; - ''') - - result = instance.query('SELECT * FROM test.kafka;') - expected = '''\ -{"t": 123, "e": {"x": "woof"} } -{"t": 124, "e": {"x": "test"} } -{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"} -''' - assert TSV(result) == TSV(expected) - assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") - -def test_kafka_json_as_string_no_kdc(kafka_cluster): - kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) - - kafka_cluster.pause_container('kafka_kerberos') - time.sleep(45) # wait for ticket expiration - - instance.query(''' - CREATE TABLE test.kafka_no_kdc (field String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kerberized_kafka1:19092', - kafka_topic_list = 'kafka_json_as_string_no_kdc', - kafka_group_name = 'kafka_json_as_string_no_kdc', - kafka_format = 'JSONAsString', - kafka_flush_interval_ms=1000; - ''') - - result = instance.query('SELECT * FROM test.kafka_no_kdc;') - expected = '' - - kafka_cluster.unpause_container('kafka_kerberos') - - - assert TSV(result) == TSV(expected) - assert instance.contains_in_log("StorageKafka (kafka_no_kdc): Nothing to commit") - assert instance.contains_in_log("Ticket expired") - assert instance.contains_in_log("Kerberos ticket refresh failed") - - -if __name__ == '__main__': - cluster.start() - raw_input("Cluster created, press any key to destroy...") - cluster.shutdown() From 9000ec0331fe062552590d5b253540e5af039b4f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 24 Sep 2020 16:21:08 +0300 Subject: [PATCH 058/131] trigger CI again to run intergation tests From 651518566a527f8a0c225f1d8a130712c8b810ee Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 24 Sep 2020 16:24:44 +0300 Subject: [PATCH 059/131] performance comparison --- docker/test/performance-comparison/perf.py | 23 ++++++++++++++++------ 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 2e047e6fb84..23686091e45 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -24,10 +24,11 @@ def tsv_escape(s): parser = argparse.ArgumentParser(description='Run performance test.') # Explicitly decode files as UTF-8 because sometimes we have Russian characters in queries, and LANG=C is set. parser.add_argument('file', metavar='FILE', type=argparse.FileType('r', encoding='utf-8'), nargs=1, help='test description file') -parser.add_argument('--host', nargs='*', default=['localhost'], help="Server hostname(s). Corresponds to '--port' options.") -parser.add_argument('--port', nargs='*', default=[9000], help="Server port(s). Corresponds to '--host' options.") +parser.add_argument('--host', nargs='*', default=['localhost'], help="Space-separated list of server hostname(s). Corresponds to '--port' options.") +parser.add_argument('--port', nargs='*', default=[9000], help="Space-separated list of server port(s). Corresponds to '--host' options.") parser.add_argument('--runs', type=int, default=1, help='Number of query runs per server.') parser.add_argument('--max-queries', type=int, default=None, help='Test no more than this number of queries, chosen at random.') +parser.add_argument('--queries-to-run', nargs='*', type=int, default=None, help='Space-separated list of indexes of queries to test.') parser.add_argument('--long', action='store_true', help='Do not skip the tests tagged as long.') parser.add_argument('--print-queries', action='store_true', help='Print test queries and exit.') parser.add_argument('--print-settings', action='store_true', help='Print test settings and exit.') @@ -188,10 +189,20 @@ for t in threads: for t in threads: t.join() -# Run the queries in randomized order, but preserve their indexes as specified -# in the test XML. To avoid using too much time, limit the number of queries -# we run per test. -queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries or len(test_queries))) +queries_to_run = range(0, len(test_queries)) + +if args.max_queries: + # If specified, test a limited number of queries chosen at random. + queries_to_run = random.sample(range(0, len(test_queries)), min(len(test_queries), args.max_queries)) + +if args.queries_to_run: + # Run the specified queries, with some sanity check. + for i in args.queries_to_run: + if i < 0 or i >= len(test_queries): + print(f'There is no query no. "{i}" in this test, only [{0}-{len(test_queries) - 1}] are present') + exit(1) + + queries_to_run = args.queries_to_run # Run test queries. for query_index in queries_to_run: From dce6a436f37713224c242092446e1a9fcf8b4a4a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 24 Sep 2020 19:35:17 +0300 Subject: [PATCH 060/131] Fix strange code --- src/Common/randomSeed.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Common/randomSeed.cpp b/src/Common/randomSeed.cpp index 8ad624febdd..ded224e56c3 100644 --- a/src/Common/randomSeed.cpp +++ b/src/Common/randomSeed.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -19,7 +20,7 @@ namespace DB DB::UInt64 randomSeed() { struct timespec times; - if (clock_gettime(CLOCK_THREAD_CPUTIME_ID, ×)) + if (clock_gettime(CLOCK_MONOTONIC, ×)) DB::throwFromErrno("Cannot clock_gettime.", DB::ErrorCodes::CANNOT_CLOCK_GETTIME); /// Not cryptographically secure as time, pid and stack address can be predictable. @@ -27,7 +28,7 @@ DB::UInt64 randomSeed() SipHash hash; hash.update(times.tv_nsec); hash.update(times.tv_sec); - hash.update(getpid()); + hash.update(getThreadId()); hash.update(×); return hash.get64(); } From 4ed4205647fe08613275a70e14aafc8e9180d8be Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 24 Sep 2020 19:54:09 +0300 Subject: [PATCH 061/131] Update run.sh --- docker/test/fasttest/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 7f3a102276e..39992c072e3 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -97,7 +97,7 @@ ccache --zero-stats ||: mkdir build cd build cmake .. -DCMAKE_INSTALL_PREFIX=/usr -DCMAKE_CXX_COMPILER=clang++-10 -DCMAKE_C_COMPILER=clang-10 "${CMAKE_LIBS_CONFIG[@]}" "${FASTTEST_CMAKE_FLAGS[@]}" | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/cmake_log.txt -ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/build_log.txt +time ninja clickhouse-bundle | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/build_log.txt ninja install | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/install_log.txt @@ -192,7 +192,7 @@ TESTS_TO_SKIP=( 01460_DistributedFilesToInsert ) -clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt +time clickhouse-test -j 8 --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_output/test_log.txt # substr is to remove semicolon after test name @@ -210,7 +210,7 @@ then kill_clickhouse # Clean the data so that there is no interference from the previous test run. - rm -rvf /var/lib/clickhouse ||: + rm -rf /var/lib/clickhouse ||: mkdir /var/lib/clickhouse clickhouse-server --config /etc/clickhouse-server/config.xml --daemon From ab01fb36da20562bf41c298d31961b3917a59432 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 20:05:13 +0300 Subject: [PATCH 062/131] Update ProcessList.cpp --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d86b5678f6d..cb9a75f26ba 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -431,7 +431,7 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even } if (get_settings && query_context) - res.query_settings = std::make_shared(query_context->getSettingsRef()); + res.query_settings = std::make_shared(query_context->getSettings()); return res; } From 5ff5ca1fa6306f602aa8ba09e6884854313cc4e3 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 20:08:15 +0300 Subject: [PATCH 063/131] Update Context.cpp --- src/Interpreters/Context.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 704b21f3a4a..a3e0e306961 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -968,6 +968,7 @@ StoragePtr Context::getViewSource() Settings Context::getSettings() const { + auto lock = getLock(); return settings; } From a4a5b96d4a0915ca74c41b9c8ed7952c46a1e150 Mon Sep 17 00:00:00 2001 From: alekseik1 <1alekseik1@gmail.com> Date: Thu, 24 Sep 2020 20:08:16 +0300 Subject: [PATCH 064/131] do not expose ClickHouse server until initialization is complete --- docker/server/entrypoint.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 8fc9c670b06..ba352c2bbc2 100644 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -89,7 +89,8 @@ EOT fi if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then - $gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG & + # Listen only on localhost until the initialization is done + $gosu /usr/bin/clickhouse-server --config-file=$CLICKHOUSE_CONFIG -- --listen_host=127.0.0.1 & pid="$!" # check if clickhouse is ready to accept connections From c6ee67d3d8cac7d6524ed2c1eef89446ffcfb72b Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Thu, 24 Sep 2020 20:27:37 +0300 Subject: [PATCH 065/131] Bump CI. [2] From 0306c586a2896097938571da8adb232a444cfadc Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 21:40:50 +0300 Subject: [PATCH 066/131] Update skip_list.json --- tests/queries/skip_list.json | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index e4713b2d960..4c00602c5d8 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -97,6 +97,7 @@ "00604_show_create_database", /// UUID must be specified in ATTACH TABLE "01190_full_attach_syntax", + "01249_bad_arguments_for_bloom_filter", /// Assumes blocking DROP "01320_create_sync_race_condition", /// Internal distionary name is different From c268464f9f83d80a24e62d5bbd241e73c55dd8f6 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 24 Sep 2020 22:38:58 +0300 Subject: [PATCH 067/131] add some disabled tests from arcadia to skip_list.json --- tests/queries/skip_list.json | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 4c00602c5d8..834590a3453 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -103,5 +103,9 @@ /// Internal distionary name is different "01225_show_create_table_from_dictionary", "01224_no_superfluous_dict_reload" + ], + "polymorphic-parts": [ + "00933_test_fix_extra_seek_on_compressed_cache", + "00446_clear_column_in_partition_zookeeper" ] } From e51d9dd2e76a9c5eade12a6518ed294705598ea3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 02:24:58 +0300 Subject: [PATCH 068/131] Fix MSan report in QueryLog --- src/Interpreters/ProcessList.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d86b5678f6d..018ddbcfa1d 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -401,7 +401,7 @@ void ProcessList::killAllQueries() QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const { - QueryStatusInfo res; + QueryStatusInfo res{}; res.query = query; res.client_info = client_info; From 13529265c45ee0b2e526ecd4d6f0e0fc797a2fb5 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 25 Sep 2020 02:28:57 +0300 Subject: [PATCH 069/131] Revert "Avoid deadlocks in Log/TinyLog" --- src/Common/FileChecker.cpp | 5 -- src/Common/FileChecker.h | 4 +- src/Storages/StorageFile.cpp | 40 +++------ src/Storages/StorageFile.h | 2 +- src/Storages/StorageLog.cpp | 60 ++++--------- src/Storages/StorageLog.h | 4 +- src/Storages/StorageStripeLog.cpp | 45 +++------- src/Storages/StorageStripeLog.h | 2 +- src/Storages/StorageTinyLog.cpp | 87 ++++++------------- src/Storages/StorageTinyLog.h | 4 +- .../0_stateless/01499_log_deadlock.reference | 3 - .../0_stateless/01499_log_deadlock.sql | 26 ------ ...2_long_log_tinylog_deadlock_race.reference | 6 -- .../01502_long_log_tinylog_deadlock_race.sh | 85 ------------------ .../01505_log_distributed_deadlock.reference | 0 .../01505_log_distributed_deadlock.sql | 12 --- .../queries/0_stateless/arcadia_skip_list.txt | 1 - 17 files changed, 79 insertions(+), 307 deletions(-) delete mode 100644 tests/queries/0_stateless/01499_log_deadlock.reference delete mode 100644 tests/queries/0_stateless/01499_log_deadlock.sql delete mode 100644 tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference delete mode 100755 tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh delete mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.reference delete mode 100644 tests/queries/0_stateless/01505_log_distributed_deadlock.sql diff --git a/src/Common/FileChecker.cpp b/src/Common/FileChecker.cpp index b306c3af990..6cbec3bda77 100644 --- a/src/Common/FileChecker.cpp +++ b/src/Common/FileChecker.cpp @@ -41,11 +41,6 @@ void FileChecker::setEmpty(const String & full_file_path) map[fileName(full_file_path)] = 0; } -FileChecker::Map FileChecker::getFileSizes() const -{ - return map; -} - CheckResults FileChecker::check() const { // Read the files again every time you call `check` - so as not to violate the constancy. diff --git a/src/Common/FileChecker.h b/src/Common/FileChecker.h index 59e7331952e..015d4cadb07 100644 --- a/src/Common/FileChecker.h +++ b/src/Common/FileChecker.h @@ -27,12 +27,10 @@ public: /// The purpose of this function is to rollback a group of unfinished writes. void repair(); +private: /// File name -> size. using Map = std::map; - Map getFileSizes() const; - -private: void initialize(); void updateImpl(const String & file_path); void load(Map & local_map, const String & path) const; diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 7b094f9bc06..cc47047dc78 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -52,7 +52,6 @@ namespace ErrorCodes extern const int UNKNOWN_IDENTIFIER; extern const int INCORRECT_FILE_NAME; extern const int FILE_DOESNT_EXIST; - extern const int TIMEOUT_EXCEEDED; } namespace @@ -200,17 +199,6 @@ StorageFile::StorageFile(CommonArguments args) setInMemoryMetadata(storage_metadata); } - -static std::chrono::seconds getLockTimeout(const Context & context) -{ - const Settings & settings = context.getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - class StorageFileSource : public SourceWithProgress { public: @@ -257,9 +245,7 @@ public: { if (storage->use_table_fd) { - unique_lock = std::unique_lock(storage->rwlock, getLockTimeout(context)); - if (!unique_lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + unique_lock = std::unique_lock(storage->rwlock); /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache /// and add ability to seek unseekable files, but cache sync isn't supported. @@ -278,9 +264,7 @@ public: } else { - shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context)); - if (!shared_lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + shared_lock = std::shared_lock(storage->rwlock); } } @@ -389,8 +373,8 @@ private: bool finished_generate = false; - std::shared_lock shared_lock; - std::unique_lock unique_lock; + std::shared_lock shared_lock; + std::unique_lock unique_lock; }; @@ -433,7 +417,7 @@ Pipe StorageFile::read( for (size_t i = 0; i < num_streams; ++i) pipes.emplace_back(std::make_shared( - this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); + this_ptr, metadata_snapshot, context, max_block_size, files_info, metadata_snapshot->getColumns().getDefaults())); return Pipe::unitePipes(std::move(pipes)); } @@ -445,16 +429,12 @@ public: explicit StorageFileBlockOutputStream( StorageFile & storage_, const StorageMetadataPtr & metadata_snapshot_, - std::unique_lock && lock_, const CompressionMethod compression_method, const Context & context) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(std::move(lock_)) + , lock(storage.rwlock) { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - std::unique_ptr naked_buffer = nullptr; if (storage.use_table_fd) { @@ -508,7 +488,7 @@ public: private: StorageFile & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; std::unique_ptr write_buf; BlockOutputStreamPtr writer; bool prefix_written{false}; @@ -526,7 +506,7 @@ BlockOutputStreamPtr StorageFile::write( if (!paths.empty()) path = paths[0]; - return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}, + return std::make_shared(*this, metadata_snapshot, chooseCompressionMethod(path, compression_method), context); } @@ -549,6 +529,8 @@ void StorageFile::rename(const String & new_path_to_table_data, const StorageID if (path_new == paths[0]) return; + std::unique_lock lock(rwlock); + Poco::File(Poco::Path(path_new).parent()).createDirectories(); Poco::File(paths[0]).renameTo(path_new); @@ -565,6 +547,8 @@ void StorageFile::truncate( if (paths.size() != 1) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + std::unique_lock lock(rwlock); + if (use_table_fd) { if (0 != ::ftruncate(table_fd, 0)) diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index babc56e3a11..ea70dcd5311 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -89,7 +89,7 @@ private: std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads - mutable std::shared_timed_mutex rwlock; + mutable std::shared_mutex rwlock; Poco::Logger * log = &Poco::Logger::get("StorageFile"); }; diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index 2fbce21655c..e437bfb05f1 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -39,7 +39,6 @@ namespace DB namespace ErrorCodes { - extern const int TIMEOUT_EXCEEDED; extern const int LOGICAL_ERROR; extern const int DUPLICATE_COLUMN; extern const int SIZES_OF_MARKS_FILES_ARE_INCONSISTENT; @@ -51,6 +50,7 @@ namespace ErrorCodes class LogSource final : public SourceWithProgress { public: + static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -116,16 +116,13 @@ private: class LogBlockOutputStream final : public IBlockOutputStream { public: - explicit LogBlockOutputStream( - StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) + explicit LogBlockOutputStream(StorageLog & storage_, const StorageMetadataPtr & metadata_snapshot_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(std::move(lock_)) + , lock(storage.rwlock) , marks_stream( storage.disk->writeFile(storage.marks_file_path, 4096, WriteMode::Rewrite)) { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~LogBlockOutputStream() override @@ -152,7 +149,7 @@ public: private: StorageLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -510,11 +507,9 @@ void StorageLog::addFiles(const String & column_name, const IDataType & type) } -void StorageLog::loadMarks(std::chrono::seconds lock_timeout) +void StorageLog::loadMarks() { - std::unique_lock lock(rwlock, lock_timeout); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + std::unique_lock lock(rwlock); if (loaded_marks) return; @@ -557,6 +552,8 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & { assert(table_path != new_path_to_table_data); { + std::unique_lock lock(rwlock); + disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -572,6 +569,8 @@ void StorageLog::rename(const String & new_path_to_table_data, const StorageID & void StorageLog::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { + std::shared_lock lock(rwlock); + files.clear(); file_count = 0; loaded_marks = false; @@ -611,17 +610,6 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount(const StorageMeta return it->second.marks; } - -static std::chrono::seconds getLockTimeout(const Context & context) -{ - const Settings & settings = context.getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - Pipe StorageLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -632,15 +620,11 @@ Pipe StorageLog::read( unsigned num_streams) { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - - auto lock_timeout = getLockTimeout(context); - loadMarks(lock_timeout); + loadMarks(); NamesAndTypesList all_columns = Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)); - std::shared_lock lock(rwlock, lock_timeout); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + std::shared_lock lock(rwlock); Pipes pipes; @@ -669,28 +653,18 @@ Pipe StorageLog::read( max_read_buffer_size)); } - /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe::unitePipes(std::move(pipes)); } -BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) +BlockOutputStreamPtr StorageLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - auto lock_timeout = getLockTimeout(context); - loadMarks(lock_timeout); - - std::unique_lock lock(rwlock, lock_timeout); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - return std::make_shared(*this, metadata_snapshot, std::move(lock)); + loadMarks(); + return std::make_shared(*this, metadata_snapshot); } -CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & context) +CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & /* context */) { - std::shared_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - + std::shared_lock lock(rwlock); return file_checker.check(); } diff --git a/src/Storages/StorageLog.h b/src/Storages/StorageLog.h index 3553426b9e6..49fc9a576c5 100644 --- a/src/Storages/StorageLog.h +++ b/src/Storages/StorageLog.h @@ -83,7 +83,7 @@ private: DiskPtr disk; String table_path; - mutable std::shared_timed_mutex rwlock; + mutable std::shared_mutex rwlock; Files files; @@ -104,7 +104,7 @@ private: /// Read marks files if they are not already read. /// It is done lazily, so that with a large number of tables, the server starts quickly. /// You can not call with a write locked `rwlock`. - void loadMarks(std::chrono::seconds lock_timeout); + void loadMarks(); /** For normal columns, the number of rows in the block is specified in the marks. * For array columns and nested structures, there are more than one group of marks that correspond to different files diff --git a/src/Storages/StorageStripeLog.cpp b/src/Storages/StorageStripeLog.cpp index 8ff8035c128..c4344cf6f1f 100644 --- a/src/Storages/StorageStripeLog.cpp +++ b/src/Storages/StorageStripeLog.cpp @@ -47,13 +47,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_FILE_NAME; - extern const int TIMEOUT_EXCEEDED; } class StripeLogSource final : public SourceWithProgress { public: + static Block getHeader( StorageStripeLog & storage, const StorageMetadataPtr & metadata_snapshot, @@ -157,11 +157,10 @@ private: class StripeLogBlockOutputStream final : public IBlockOutputStream { public: - explicit StripeLogBlockOutputStream( - StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_, std::unique_lock && lock_) + explicit StripeLogBlockOutputStream(StorageStripeLog & storage_, const StorageMetadataPtr & metadata_snapshot_) : storage(storage_) , metadata_snapshot(metadata_snapshot_) - , lock(std::move(lock_)) + , lock(storage.rwlock) , data_out_file(storage.table_path + "data.bin") , data_out_compressed(storage.disk->writeFile(data_out_file, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Append)) , data_out(std::make_unique( @@ -171,8 +170,6 @@ public: , index_out(std::make_unique(*index_out_compressed)) , block_out(*data_out, 0, metadata_snapshot->getSampleBlock(), false, index_out.get(), storage.disk->getFileSize(data_out_file)) { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~StripeLogBlockOutputStream() override @@ -226,7 +223,7 @@ public: private: StorageStripeLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; String data_out_file; std::unique_ptr data_out_compressed; @@ -289,6 +286,8 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora { assert(table_path != new_path_to_table_data); { + std::unique_lock lock(rwlock); + disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -298,16 +297,6 @@ void StorageStripeLog::rename(const String & new_path_to_table_data, const Stora } -static std::chrono::seconds getLockTimeout(const Context & context) -{ - const Settings & settings = context.getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - Pipe StorageStripeLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -317,9 +306,7 @@ Pipe StorageStripeLog::read( const size_t /*max_block_size*/, unsigned num_streams) { - std::shared_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); + std::shared_lock lock(rwlock); metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); @@ -358,28 +345,24 @@ Pipe StorageStripeLog::read( } -BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) +BlockOutputStreamPtr StorageStripeLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - std::unique_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - return std::make_shared(*this, metadata_snapshot, std::move(lock)); + return std::make_shared(*this, metadata_snapshot); } -CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & context) +CheckResults StorageStripeLog::checkData(const ASTPtr & /* query */, const Context & /* context */) { - std::shared_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - + std::shared_lock lock(rwlock); return file_checker.check(); } void StorageStripeLog::truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) { + std::shared_lock lock(rwlock); + disk->clearDirectory(table_path); + file_checker = FileChecker{disk, table_path + "sizes.json"}; } diff --git a/src/Storages/StorageStripeLog.h b/src/Storages/StorageStripeLog.h index ca3bfe4ff75..f88120a932e 100644 --- a/src/Storages/StorageStripeLog.h +++ b/src/Storages/StorageStripeLog.h @@ -67,7 +67,7 @@ private: size_t max_compress_block_size; FileChecker file_checker; - mutable std::shared_timed_mutex rwlock; + mutable std::shared_mutex rwlock; Poco::Logger * log; }; diff --git a/src/Storages/StorageTinyLog.cpp b/src/Storages/StorageTinyLog.cpp index 4d646c7451e..0bdcab8abf4 100644 --- a/src/Storages/StorageTinyLog.cpp +++ b/src/Storages/StorageTinyLog.cpp @@ -13,7 +13,6 @@ #include #include -#include #include #include #include @@ -47,7 +46,6 @@ namespace DB namespace ErrorCodes { - extern const int TIMEOUT_EXCEEDED; extern const int DUPLICATE_COLUMN; extern const int INCORRECT_FILE_NAME; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; @@ -57,6 +55,7 @@ namespace ErrorCodes class TinyLogSource final : public SourceWithProgress { public: + static Block getHeader(const NamesAndTypesList & columns) { Block res; @@ -67,17 +66,10 @@ public: return Nested::flatten(res); } - TinyLogSource( - size_t block_size_, - const NamesAndTypesList & columns_, - StorageTinyLog & storage_, - size_t max_read_buffer_size_, - FileChecker::Map file_sizes_) + TinyLogSource(size_t block_size_, const NamesAndTypesList & columns_, StorageTinyLog & storage_, size_t max_read_buffer_size_) : SourceWithProgress(getHeader(columns_)) - , block_size(block_size_), columns(columns_), storage(storage_) - , max_read_buffer_size(max_read_buffer_size_), file_sizes(std::move(file_sizes_)) - { - } + , block_size(block_size_), columns(columns_), storage(storage_), lock(storage_.rwlock) + , max_read_buffer_size(max_read_buffer_size_) {} String getName() const override { return "TinyLog"; } @@ -88,21 +80,19 @@ private: size_t block_size; NamesAndTypesList columns; StorageTinyLog & storage; + std::shared_lock lock; bool is_finished = false; size_t max_read_buffer_size; - FileChecker::Map file_sizes; struct Stream { - Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_, size_t file_size) + Stream(const DiskPtr & disk, const String & data_path, size_t max_read_buffer_size_) : plain(disk->readFile(data_path, std::min(max_read_buffer_size_, disk->getFileSize(data_path)))), - limited(std::make_unique(*plain, file_size, false)), compressed(*plain) { } std::unique_ptr plain; - std::unique_ptr limited; CompressedReadBuffer compressed; }; @@ -120,14 +110,9 @@ private: class TinyLogBlockOutputStream final : public IBlockOutputStream { public: - explicit TinyLogBlockOutputStream( - StorageTinyLog & storage_, - const StorageMetadataPtr & metadata_snapshot_, - std::unique_lock && lock_) - : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(std::move(lock_)) + explicit TinyLogBlockOutputStream(StorageTinyLog & storage_, const StorageMetadataPtr & metadata_snapshot_) + : storage(storage_), metadata_snapshot(metadata_snapshot_), lock(storage_.rwlock) { - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); } ~TinyLogBlockOutputStream() override @@ -155,7 +140,7 @@ public: private: StorageTinyLog & storage; StorageMetadataPtr metadata_snapshot; - std::unique_lock lock; + std::unique_lock lock; bool done = false; struct Stream @@ -246,17 +231,13 @@ void TinyLogSource::readData(const String & name, const IDataType & type, IColum String stream_name = IDataType::getFileNameForStream(name, path); if (!streams.count(stream_name)) - { - String file_path = storage.files[stream_name].data_file_path; - streams[stream_name] = std::make_unique( - storage.disk, file_path, max_read_buffer_size, file_sizes[fileName(file_path)]); - } + streams[stream_name] = std::make_unique(storage.disk, storage.files[stream_name].data_file_path, max_read_buffer_size); return &streams[stream_name]->compressed; }; if (deserialize_states.count(name) == 0) - type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); + type.deserializeBinaryBulkStatePrefix(settings, deserialize_states[name]); type.deserializeBinaryBulkWithMultipleStreams(column, limit, settings, deserialize_states[name]); } @@ -429,6 +410,8 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage { assert(table_path != new_path_to_table_data); { + std::unique_lock lock(rwlock); + disk->moveDirectory(table_path, new_path_to_table_data); table_path = new_path_to_table_data; @@ -441,16 +424,6 @@ void StorageTinyLog::rename(const String & new_path_to_table_data, const Storage } -static std::chrono::seconds getLockTimeout(const Context & context) -{ - const Settings & settings = context.getSettingsRef(); - Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds(); - if (settings.max_execution_time.totalSeconds() != 0 && settings.max_execution_time.totalSeconds() < lock_timeout) - lock_timeout = settings.max_execution_time.totalSeconds(); - return std::chrono::seconds{lock_timeout}; -} - - Pipe StorageTinyLog::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -464,40 +437,28 @@ Pipe StorageTinyLog::read( // When reading, we lock the entire storage, because we only have one file // per column and can't modify it concurrently. - const Settings & settings = context.getSettingsRef(); - - std::shared_lock lock{rwlock, getLockTimeout(context)}; - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - - /// No need to hold lock while reading because we read fixed range of data that does not change while appending more data. return Pipe(std::make_shared( - max_block_size, - Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), - *this, - settings.max_read_buffer_size, - file_checker.getFileSizes())); + max_block_size, Nested::collect(metadata_snapshot->getColumns().getAllPhysical().addTypes(column_names)), *this, context.getSettingsRef().max_read_buffer_size)); } -BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & context) +BlockOutputStreamPtr StorageTinyLog::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, const Context & /*context*/) { - return std::make_shared(*this, metadata_snapshot, std::unique_lock{rwlock, getLockTimeout(context)}); + return std::make_shared(*this, metadata_snapshot); } -CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & context) +CheckResults StorageTinyLog::checkData(const ASTPtr & /* query */, const Context & /* context */) { - std::shared_lock lock(rwlock, getLockTimeout(context)); - if (!lock) - throw Exception("Lock timeout exceeded", ErrorCodes::TIMEOUT_EXCEEDED); - + std::shared_lock lock(rwlock); return file_checker.check(); } void StorageTinyLog::truncate( const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) { + std::unique_lock lock(rwlock); + disk->clearDirectory(table_path); files.clear(); @@ -507,6 +468,14 @@ void StorageTinyLog::truncate( addFiles(column.name, *column.type); } +void StorageTinyLog::drop() +{ + std::unique_lock lock(rwlock); + if (disk->exists(table_path)) + disk->removeRecursive(table_path); + files.clear(); +} + void registerStorageTinyLog(StorageFactory & factory) { diff --git a/src/Storages/StorageTinyLog.h b/src/Storages/StorageTinyLog.h index 95b7d9f2941..dc6ff101503 100644 --- a/src/Storages/StorageTinyLog.h +++ b/src/Storages/StorageTinyLog.h @@ -43,6 +43,8 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; + void drop() override; + protected: StorageTinyLog( DiskPtr disk_, @@ -68,7 +70,7 @@ private: Files files; FileChecker file_checker; - mutable std::shared_timed_mutex rwlock; + mutable std::shared_mutex rwlock; Poco::Logger * log; diff --git a/tests/queries/0_stateless/01499_log_deadlock.reference b/tests/queries/0_stateless/01499_log_deadlock.reference deleted file mode 100644 index 166be640db5..00000000000 --- a/tests/queries/0_stateless/01499_log_deadlock.reference +++ /dev/null @@ -1,3 +0,0 @@ -6 -6 -6 diff --git a/tests/queries/0_stateless/01499_log_deadlock.sql b/tests/queries/0_stateless/01499_log_deadlock.sql deleted file mode 100644 index e98b37f2455..00000000000 --- a/tests/queries/0_stateless/01499_log_deadlock.sql +++ /dev/null @@ -1,26 +0,0 @@ -DROP TABLE IF EXISTS t; -CREATE TABLE t (x UInt8) ENGINE = TinyLog; - -INSERT INTO t VALUES (1), (2), (3); -INSERT INTO t SELECT * FROM t; -SELECT count() FROM t; - -DROP TABLE t; - - -CREATE TABLE t (x UInt8) ENGINE = Log; - -INSERT INTO t VALUES (1), (2), (3); -INSERT INTO t SELECT * FROM t; -SELECT count() FROM t; - -DROP TABLE t; - - -CREATE TABLE t (x UInt8) ENGINE = StripeLog; - -INSERT INTO t VALUES (1), (2), (3); -INSERT INTO t SELECT * FROM t; -SELECT count() FROM t; - -DROP TABLE t; diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference deleted file mode 100644 index 4bf85ae79f3..00000000000 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.reference +++ /dev/null @@ -1,6 +0,0 @@ -Testing TinyLog -Done TinyLog -Testing StripeLog -Done StripeLog -Testing Log -Done Log diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh deleted file mode 100755 index 29c5f868617..00000000000 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh +++ /dev/null @@ -1,85 +0,0 @@ -#!/usr/bin/env bash - -set -e - -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. "$CURDIR"/../shell_config.sh - - -function thread_create { - while true; do - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM - done -} - -function thread_drop { - while true; do - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM - done -} - -function thread_rename { - while true; do - $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM - done -} - -function thread_select { - while true; do - $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -function thread_insert { - while true; do - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -function thread_insert_select { - while true; do - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM - done -} - -export -f thread_create -export -f thread_drop -export -f thread_rename -export -f thread_select -export -f thread_insert -export -f thread_insert_select - - -# Do randomized queries and expect nothing extraordinary happens. - -function test_with_engine { - echo "Testing $1" - - timeout 10 bash -c "thread_create t1 $1" & - timeout 10 bash -c "thread_create t2 $1" & - timeout 10 bash -c 'thread_drop t1' & - timeout 10 bash -c 'thread_drop t2' & - timeout 10 bash -c 'thread_rename t1 t2' & - timeout 10 bash -c 'thread_rename t2 t1' & - timeout 10 bash -c 'thread_select t1' & - timeout 10 bash -c 'thread_select t2' & - timeout 10 bash -c 'thread_insert t1 5' & - timeout 10 bash -c 'thread_insert t2 10' & - timeout 10 bash -c 'thread_insert_select t1 t2' & - timeout 10 bash -c 'thread_insert_select t2 t1' & - - wait - echo "Done $1" -} - -test_with_engine TinyLog -test_with_engine StripeLog -test_with_engine Log diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.reference b/tests/queries/0_stateless/01505_log_distributed_deadlock.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql b/tests/queries/0_stateless/01505_log_distributed_deadlock.sql deleted file mode 100644 index 2b0b2b97188..00000000000 --- a/tests/queries/0_stateless/01505_log_distributed_deadlock.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS t_local; -DROP TABLE IF EXISTS t_dist; - -create table t_local(a int) engine Log; -create table t_dist (a int) engine Distributed(test_shard_localhost, currentDatabase(), 't_local', cityHash64(a)); - -set insert_distributed_sync = 1; - -insert into t_dist values (1); - -DROP TABLE t_local; -DROP TABLE t_dist; diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6d1c6444d1b..69391ca9fd4 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -145,4 +145,3 @@ 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read 01505_distributed_local_type_conversion_enum -01505_log_distributed_deadlock From fb9d43692bcdd435377db0380e79dc33e47b6d7f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 03:10:37 +0300 Subject: [PATCH 070/131] Step 1: make adding hardware benchmark results more convenient --- .../results/{022_amd_epyc_7402p.json => amd_epyc_7402p.json} | 0 .../results/{043_amd_epyc_7502p.json => amd_epyc_7502p.json} | 0 .../results/{005_amd_epyc_7551.json => amd_epyc_7551.json} | 0 .../results/{052_amd_epyc_7642.json => amd_epyc_7642.json} | 0 .../results/{041_amd_epyc_7702.json => amd_epyc_7702.json} | 0 .../{038_amd_ryzen_9_3950x.json => amd_ryzen_9_3950x.json} | 0 .../results/{035_aws_a1_4xlarge.json => aws_a1_4xlarge.json} | 0 .../hardware/results/{049_aws_c5metal.json => aws_c5metal.json} | 0 .../results/{015_aws_i3_8xlarge.json => aws_i3_8xlarge.json} | 0 .../{017_aws_i3en_24xlarge.json => aws_i3en_24xlarge.json} | 0 .../{046_aws_lightsail_4vcpu.json => aws_lightsail_4vcpu.json} | 0 .../results/{051_aws_m5a_8xlarge.json => aws_m5a_8xlarge.json} | 0 .../{019_aws_m5ad_24xlarge.json => aws_m5ad_24xlarge.json} | 0 .../results/{016_aws_m5d_24xlarge.json => aws_m5d_24xlarge.json} | 0 .../results/{045_aws_m6g_16xlarge.json => aws_m6g_16xlarge.json} | 0 .../hardware/results/{014_azure_ds3v2.json => azure_ds3v2.json} | 0 .../hardware/results/{039_azure_e32s.json => azure_e32s.json} | 0 ...e_i5_3210M_lenovo_b580.json => core_i5_3210M_lenovo_b580.json} | 0 ...ore_i7_6770hq_intel_nuc.json => core_i7_6770hq_intel_nuc.json} | 0 ..._core_i7_8550u_lenovo_x1.json => core_i7_8550u_lenovo_x1.json} | 0 ...ore_i7_macbook_pro_2018.json => core_i7_macbook_pro_2018.json} | 0 .../hardware/results/{012_dell_r530.json => dell_r530.json} | 0 .../hardware/results/{047_dell_xps.json => dell_xps.json} | 0 ...18_huawei_taishan_2280_v2.json => huawei_taishan_2280_v2.json} | 0 .../hardware/results/{037_pinebook_pro.json => pinebook_pro.json} | 0 .../hardware/results/{048_pixel_3a.json => pixel_3a.json} | 0 ...{024_selectel_cloud_16vcpu.json => selectel_cloud_16vcpu.json} | 0 .../hardware/results/{008_skylake_kvm.json => skylake_kvm.json} | 0 .../hardware/results/{013_xeon_2176g.json => xeon_2176g.json} | 0 .../hardware/results/{021_xeon_e5645.json => xeon_e5645.json} | 0 .../results/{023_xeon_e5_1650v3.json => xeon_e5_1650v3.json} | 0 .../results/{010_xeon_e5_2640v4.json => xeon_e5_2640v4.json} | 0 .../hardware/results/{007_xeon_e5_2650.json => xeon_e5_2650.json} | 0 .../results/{050_xeon_e5_2650l_v3.json => xeon_e5_2650l_v3.json} | 0 .../results/{001_xeon_gold_6230.json => xeon_gold_6230.json} | 0 .../results/{044_xeon_silver_4114.json => xeon_silver_4114.json} | 0 .../hardware/results/{006_xeon_sp_gold.json => xeon_sp_gold.json} | 0 .../hardware/results/{036_xeon_x5675.json => xeon_x5675.json} | 0 ...d_broadwell_4_vcpu.json => yandex_cloud_broadwell_4_vcpu.json} | 0 ...ade_lake_4_vcpu.json => yandex_cloud_cascade_lake_4_vcpu.json} | 0 ...e_lake_64_vcpu.json => yandex_cloud_cascade_lake_64_vcpu.json} | 0 ..._s3_3xlarge.json => yandex_managed_clickhouse_s3_3xlarge.json} | 0 42 files changed, 0 insertions(+), 0 deletions(-) rename website/benchmark/hardware/results/{022_amd_epyc_7402p.json => amd_epyc_7402p.json} (100%) rename website/benchmark/hardware/results/{043_amd_epyc_7502p.json => amd_epyc_7502p.json} (100%) rename website/benchmark/hardware/results/{005_amd_epyc_7551.json => amd_epyc_7551.json} (100%) rename website/benchmark/hardware/results/{052_amd_epyc_7642.json => amd_epyc_7642.json} (100%) rename website/benchmark/hardware/results/{041_amd_epyc_7702.json => amd_epyc_7702.json} (100%) rename website/benchmark/hardware/results/{038_amd_ryzen_9_3950x.json => amd_ryzen_9_3950x.json} (100%) rename website/benchmark/hardware/results/{035_aws_a1_4xlarge.json => aws_a1_4xlarge.json} (100%) rename website/benchmark/hardware/results/{049_aws_c5metal.json => aws_c5metal.json} (100%) rename website/benchmark/hardware/results/{015_aws_i3_8xlarge.json => aws_i3_8xlarge.json} (100%) rename website/benchmark/hardware/results/{017_aws_i3en_24xlarge.json => aws_i3en_24xlarge.json} (100%) rename website/benchmark/hardware/results/{046_aws_lightsail_4vcpu.json => aws_lightsail_4vcpu.json} (100%) rename website/benchmark/hardware/results/{051_aws_m5a_8xlarge.json => aws_m5a_8xlarge.json} (100%) rename website/benchmark/hardware/results/{019_aws_m5ad_24xlarge.json => aws_m5ad_24xlarge.json} (100%) rename website/benchmark/hardware/results/{016_aws_m5d_24xlarge.json => aws_m5d_24xlarge.json} (100%) rename website/benchmark/hardware/results/{045_aws_m6g_16xlarge.json => aws_m6g_16xlarge.json} (100%) rename website/benchmark/hardware/results/{014_azure_ds3v2.json => azure_ds3v2.json} (100%) rename website/benchmark/hardware/results/{039_azure_e32s.json => azure_e32s.json} (100%) rename website/benchmark/hardware/results/{009_core_i5_3210M_lenovo_b580.json => core_i5_3210M_lenovo_b580.json} (100%) rename website/benchmark/hardware/results/{042_core_i7_6770hq_intel_nuc.json => core_i7_6770hq_intel_nuc.json} (100%) rename website/benchmark/hardware/results/{020_core_i7_8550u_lenovo_x1.json => core_i7_8550u_lenovo_x1.json} (100%) rename website/benchmark/hardware/results/{040_core_i7_macbook_pro_2018.json => core_i7_macbook_pro_2018.json} (100%) rename website/benchmark/hardware/results/{012_dell_r530.json => dell_r530.json} (100%) rename website/benchmark/hardware/results/{047_dell_xps.json => dell_xps.json} (100%) rename website/benchmark/hardware/results/{018_huawei_taishan_2280_v2.json => huawei_taishan_2280_v2.json} (100%) rename website/benchmark/hardware/results/{037_pinebook_pro.json => pinebook_pro.json} (100%) rename website/benchmark/hardware/results/{048_pixel_3a.json => pixel_3a.json} (100%) rename website/benchmark/hardware/results/{024_selectel_cloud_16vcpu.json => selectel_cloud_16vcpu.json} (100%) rename website/benchmark/hardware/results/{008_skylake_kvm.json => skylake_kvm.json} (100%) rename website/benchmark/hardware/results/{013_xeon_2176g.json => xeon_2176g.json} (100%) rename website/benchmark/hardware/results/{021_xeon_e5645.json => xeon_e5645.json} (100%) rename website/benchmark/hardware/results/{023_xeon_e5_1650v3.json => xeon_e5_1650v3.json} (100%) rename website/benchmark/hardware/results/{010_xeon_e5_2640v4.json => xeon_e5_2640v4.json} (100%) rename website/benchmark/hardware/results/{007_xeon_e5_2650.json => xeon_e5_2650.json} (100%) rename website/benchmark/hardware/results/{050_xeon_e5_2650l_v3.json => xeon_e5_2650l_v3.json} (100%) rename website/benchmark/hardware/results/{001_xeon_gold_6230.json => xeon_gold_6230.json} (100%) rename website/benchmark/hardware/results/{044_xeon_silver_4114.json => xeon_silver_4114.json} (100%) rename website/benchmark/hardware/results/{006_xeon_sp_gold.json => xeon_sp_gold.json} (100%) rename website/benchmark/hardware/results/{036_xeon_x5675.json => xeon_x5675.json} (100%) rename website/benchmark/hardware/results/{004_yandex_cloud_broadwell_4_vcpu.json => yandex_cloud_broadwell_4_vcpu.json} (100%) rename website/benchmark/hardware/results/{003_yandex_cloud_cascade_lake_4_vcpu.json => yandex_cloud_cascade_lake_4_vcpu.json} (100%) rename website/benchmark/hardware/results/{002_yandex_cloud_cascade_lake_64_vcpu.json => yandex_cloud_cascade_lake_64_vcpu.json} (100%) rename website/benchmark/hardware/results/{011_yandex_managed_clickhouse_s3_3xlarge.json => yandex_managed_clickhouse_s3_3xlarge.json} (100%) diff --git a/website/benchmark/hardware/results/022_amd_epyc_7402p.json b/website/benchmark/hardware/results/amd_epyc_7402p.json similarity index 100% rename from website/benchmark/hardware/results/022_amd_epyc_7402p.json rename to website/benchmark/hardware/results/amd_epyc_7402p.json diff --git a/website/benchmark/hardware/results/043_amd_epyc_7502p.json b/website/benchmark/hardware/results/amd_epyc_7502p.json similarity index 100% rename from website/benchmark/hardware/results/043_amd_epyc_7502p.json rename to website/benchmark/hardware/results/amd_epyc_7502p.json diff --git a/website/benchmark/hardware/results/005_amd_epyc_7551.json b/website/benchmark/hardware/results/amd_epyc_7551.json similarity index 100% rename from website/benchmark/hardware/results/005_amd_epyc_7551.json rename to website/benchmark/hardware/results/amd_epyc_7551.json diff --git a/website/benchmark/hardware/results/052_amd_epyc_7642.json b/website/benchmark/hardware/results/amd_epyc_7642.json similarity index 100% rename from website/benchmark/hardware/results/052_amd_epyc_7642.json rename to website/benchmark/hardware/results/amd_epyc_7642.json diff --git a/website/benchmark/hardware/results/041_amd_epyc_7702.json b/website/benchmark/hardware/results/amd_epyc_7702.json similarity index 100% rename from website/benchmark/hardware/results/041_amd_epyc_7702.json rename to website/benchmark/hardware/results/amd_epyc_7702.json diff --git a/website/benchmark/hardware/results/038_amd_ryzen_9_3950x.json b/website/benchmark/hardware/results/amd_ryzen_9_3950x.json similarity index 100% rename from website/benchmark/hardware/results/038_amd_ryzen_9_3950x.json rename to website/benchmark/hardware/results/amd_ryzen_9_3950x.json diff --git a/website/benchmark/hardware/results/035_aws_a1_4xlarge.json b/website/benchmark/hardware/results/aws_a1_4xlarge.json similarity index 100% rename from website/benchmark/hardware/results/035_aws_a1_4xlarge.json rename to website/benchmark/hardware/results/aws_a1_4xlarge.json diff --git a/website/benchmark/hardware/results/049_aws_c5metal.json b/website/benchmark/hardware/results/aws_c5metal.json similarity index 100% rename from website/benchmark/hardware/results/049_aws_c5metal.json rename to website/benchmark/hardware/results/aws_c5metal.json diff --git a/website/benchmark/hardware/results/015_aws_i3_8xlarge.json b/website/benchmark/hardware/results/aws_i3_8xlarge.json similarity index 100% rename from website/benchmark/hardware/results/015_aws_i3_8xlarge.json rename to website/benchmark/hardware/results/aws_i3_8xlarge.json diff --git a/website/benchmark/hardware/results/017_aws_i3en_24xlarge.json b/website/benchmark/hardware/results/aws_i3en_24xlarge.json similarity index 100% rename from website/benchmark/hardware/results/017_aws_i3en_24xlarge.json rename to website/benchmark/hardware/results/aws_i3en_24xlarge.json diff --git a/website/benchmark/hardware/results/046_aws_lightsail_4vcpu.json b/website/benchmark/hardware/results/aws_lightsail_4vcpu.json similarity index 100% rename from website/benchmark/hardware/results/046_aws_lightsail_4vcpu.json rename to website/benchmark/hardware/results/aws_lightsail_4vcpu.json diff --git a/website/benchmark/hardware/results/051_aws_m5a_8xlarge.json b/website/benchmark/hardware/results/aws_m5a_8xlarge.json similarity index 100% rename from website/benchmark/hardware/results/051_aws_m5a_8xlarge.json rename to website/benchmark/hardware/results/aws_m5a_8xlarge.json diff --git a/website/benchmark/hardware/results/019_aws_m5ad_24xlarge.json b/website/benchmark/hardware/results/aws_m5ad_24xlarge.json similarity index 100% rename from website/benchmark/hardware/results/019_aws_m5ad_24xlarge.json rename to website/benchmark/hardware/results/aws_m5ad_24xlarge.json diff --git a/website/benchmark/hardware/results/016_aws_m5d_24xlarge.json b/website/benchmark/hardware/results/aws_m5d_24xlarge.json similarity index 100% rename from website/benchmark/hardware/results/016_aws_m5d_24xlarge.json rename to website/benchmark/hardware/results/aws_m5d_24xlarge.json diff --git a/website/benchmark/hardware/results/045_aws_m6g_16xlarge.json b/website/benchmark/hardware/results/aws_m6g_16xlarge.json similarity index 100% rename from website/benchmark/hardware/results/045_aws_m6g_16xlarge.json rename to website/benchmark/hardware/results/aws_m6g_16xlarge.json diff --git a/website/benchmark/hardware/results/014_azure_ds3v2.json b/website/benchmark/hardware/results/azure_ds3v2.json similarity index 100% rename from website/benchmark/hardware/results/014_azure_ds3v2.json rename to website/benchmark/hardware/results/azure_ds3v2.json diff --git a/website/benchmark/hardware/results/039_azure_e32s.json b/website/benchmark/hardware/results/azure_e32s.json similarity index 100% rename from website/benchmark/hardware/results/039_azure_e32s.json rename to website/benchmark/hardware/results/azure_e32s.json diff --git a/website/benchmark/hardware/results/009_core_i5_3210M_lenovo_b580.json b/website/benchmark/hardware/results/core_i5_3210M_lenovo_b580.json similarity index 100% rename from website/benchmark/hardware/results/009_core_i5_3210M_lenovo_b580.json rename to website/benchmark/hardware/results/core_i5_3210M_lenovo_b580.json diff --git a/website/benchmark/hardware/results/042_core_i7_6770hq_intel_nuc.json b/website/benchmark/hardware/results/core_i7_6770hq_intel_nuc.json similarity index 100% rename from website/benchmark/hardware/results/042_core_i7_6770hq_intel_nuc.json rename to website/benchmark/hardware/results/core_i7_6770hq_intel_nuc.json diff --git a/website/benchmark/hardware/results/020_core_i7_8550u_lenovo_x1.json b/website/benchmark/hardware/results/core_i7_8550u_lenovo_x1.json similarity index 100% rename from website/benchmark/hardware/results/020_core_i7_8550u_lenovo_x1.json rename to website/benchmark/hardware/results/core_i7_8550u_lenovo_x1.json diff --git a/website/benchmark/hardware/results/040_core_i7_macbook_pro_2018.json b/website/benchmark/hardware/results/core_i7_macbook_pro_2018.json similarity index 100% rename from website/benchmark/hardware/results/040_core_i7_macbook_pro_2018.json rename to website/benchmark/hardware/results/core_i7_macbook_pro_2018.json diff --git a/website/benchmark/hardware/results/012_dell_r530.json b/website/benchmark/hardware/results/dell_r530.json similarity index 100% rename from website/benchmark/hardware/results/012_dell_r530.json rename to website/benchmark/hardware/results/dell_r530.json diff --git a/website/benchmark/hardware/results/047_dell_xps.json b/website/benchmark/hardware/results/dell_xps.json similarity index 100% rename from website/benchmark/hardware/results/047_dell_xps.json rename to website/benchmark/hardware/results/dell_xps.json diff --git a/website/benchmark/hardware/results/018_huawei_taishan_2280_v2.json b/website/benchmark/hardware/results/huawei_taishan_2280_v2.json similarity index 100% rename from website/benchmark/hardware/results/018_huawei_taishan_2280_v2.json rename to website/benchmark/hardware/results/huawei_taishan_2280_v2.json diff --git a/website/benchmark/hardware/results/037_pinebook_pro.json b/website/benchmark/hardware/results/pinebook_pro.json similarity index 100% rename from website/benchmark/hardware/results/037_pinebook_pro.json rename to website/benchmark/hardware/results/pinebook_pro.json diff --git a/website/benchmark/hardware/results/048_pixel_3a.json b/website/benchmark/hardware/results/pixel_3a.json similarity index 100% rename from website/benchmark/hardware/results/048_pixel_3a.json rename to website/benchmark/hardware/results/pixel_3a.json diff --git a/website/benchmark/hardware/results/024_selectel_cloud_16vcpu.json b/website/benchmark/hardware/results/selectel_cloud_16vcpu.json similarity index 100% rename from website/benchmark/hardware/results/024_selectel_cloud_16vcpu.json rename to website/benchmark/hardware/results/selectel_cloud_16vcpu.json diff --git a/website/benchmark/hardware/results/008_skylake_kvm.json b/website/benchmark/hardware/results/skylake_kvm.json similarity index 100% rename from website/benchmark/hardware/results/008_skylake_kvm.json rename to website/benchmark/hardware/results/skylake_kvm.json diff --git a/website/benchmark/hardware/results/013_xeon_2176g.json b/website/benchmark/hardware/results/xeon_2176g.json similarity index 100% rename from website/benchmark/hardware/results/013_xeon_2176g.json rename to website/benchmark/hardware/results/xeon_2176g.json diff --git a/website/benchmark/hardware/results/021_xeon_e5645.json b/website/benchmark/hardware/results/xeon_e5645.json similarity index 100% rename from website/benchmark/hardware/results/021_xeon_e5645.json rename to website/benchmark/hardware/results/xeon_e5645.json diff --git a/website/benchmark/hardware/results/023_xeon_e5_1650v3.json b/website/benchmark/hardware/results/xeon_e5_1650v3.json similarity index 100% rename from website/benchmark/hardware/results/023_xeon_e5_1650v3.json rename to website/benchmark/hardware/results/xeon_e5_1650v3.json diff --git a/website/benchmark/hardware/results/010_xeon_e5_2640v4.json b/website/benchmark/hardware/results/xeon_e5_2640v4.json similarity index 100% rename from website/benchmark/hardware/results/010_xeon_e5_2640v4.json rename to website/benchmark/hardware/results/xeon_e5_2640v4.json diff --git a/website/benchmark/hardware/results/007_xeon_e5_2650.json b/website/benchmark/hardware/results/xeon_e5_2650.json similarity index 100% rename from website/benchmark/hardware/results/007_xeon_e5_2650.json rename to website/benchmark/hardware/results/xeon_e5_2650.json diff --git a/website/benchmark/hardware/results/050_xeon_e5_2650l_v3.json b/website/benchmark/hardware/results/xeon_e5_2650l_v3.json similarity index 100% rename from website/benchmark/hardware/results/050_xeon_e5_2650l_v3.json rename to website/benchmark/hardware/results/xeon_e5_2650l_v3.json diff --git a/website/benchmark/hardware/results/001_xeon_gold_6230.json b/website/benchmark/hardware/results/xeon_gold_6230.json similarity index 100% rename from website/benchmark/hardware/results/001_xeon_gold_6230.json rename to website/benchmark/hardware/results/xeon_gold_6230.json diff --git a/website/benchmark/hardware/results/044_xeon_silver_4114.json b/website/benchmark/hardware/results/xeon_silver_4114.json similarity index 100% rename from website/benchmark/hardware/results/044_xeon_silver_4114.json rename to website/benchmark/hardware/results/xeon_silver_4114.json diff --git a/website/benchmark/hardware/results/006_xeon_sp_gold.json b/website/benchmark/hardware/results/xeon_sp_gold.json similarity index 100% rename from website/benchmark/hardware/results/006_xeon_sp_gold.json rename to website/benchmark/hardware/results/xeon_sp_gold.json diff --git a/website/benchmark/hardware/results/036_xeon_x5675.json b/website/benchmark/hardware/results/xeon_x5675.json similarity index 100% rename from website/benchmark/hardware/results/036_xeon_x5675.json rename to website/benchmark/hardware/results/xeon_x5675.json diff --git a/website/benchmark/hardware/results/004_yandex_cloud_broadwell_4_vcpu.json b/website/benchmark/hardware/results/yandex_cloud_broadwell_4_vcpu.json similarity index 100% rename from website/benchmark/hardware/results/004_yandex_cloud_broadwell_4_vcpu.json rename to website/benchmark/hardware/results/yandex_cloud_broadwell_4_vcpu.json diff --git a/website/benchmark/hardware/results/003_yandex_cloud_cascade_lake_4_vcpu.json b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_4_vcpu.json similarity index 100% rename from website/benchmark/hardware/results/003_yandex_cloud_cascade_lake_4_vcpu.json rename to website/benchmark/hardware/results/yandex_cloud_cascade_lake_4_vcpu.json diff --git a/website/benchmark/hardware/results/002_yandex_cloud_cascade_lake_64_vcpu.json b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_64_vcpu.json similarity index 100% rename from website/benchmark/hardware/results/002_yandex_cloud_cascade_lake_64_vcpu.json rename to website/benchmark/hardware/results/yandex_cloud_cascade_lake_64_vcpu.json diff --git a/website/benchmark/hardware/results/011_yandex_managed_clickhouse_s3_3xlarge.json b/website/benchmark/hardware/results/yandex_managed_clickhouse_s3_3xlarge.json similarity index 100% rename from website/benchmark/hardware/results/011_yandex_managed_clickhouse_s3_3xlarge.json rename to website/benchmark/hardware/results/yandex_managed_clickhouse_s3_3xlarge.json From 8e986f81b441822884c801e0763798dc03ee710f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 03:24:24 +0300 Subject: [PATCH 071/131] Added new results --- ...{aws_c5metal.json => aws_c5metal_100.json} | 2 +- .../hardware/results/aws_c5metal_300.json | 54 ++++++++++++++++++ .../hardware/results/aws_c6metal.json | 54 ++++++++++++++++++ .../yandex_cloud_cascade_lake_32_vcpu.json | 55 +++++++++++++++++++ .../yandex_cloud_cascade_lake_80_vcpu.json | 55 +++++++++++++++++++ 5 files changed, 219 insertions(+), 1 deletion(-) rename website/benchmark/hardware/results/{aws_c5metal.json => aws_c5metal_100.json} (97%) create mode 100644 website/benchmark/hardware/results/aws_c5metal_300.json create mode 100644 website/benchmark/hardware/results/aws_c6metal.json create mode 100644 website/benchmark/hardware/results/yandex_cloud_cascade_lake_32_vcpu.json create mode 100644 website/benchmark/hardware/results/yandex_cloud_cascade_lake_80_vcpu.json diff --git a/website/benchmark/hardware/results/aws_c5metal.json b/website/benchmark/hardware/results/aws_c5metal_100.json similarity index 97% rename from website/benchmark/hardware/results/aws_c5metal.json rename to website/benchmark/hardware/results/aws_c5metal_100.json index 9d933500ad1..4bb0a1f1f52 100644 --- a/website/benchmark/hardware/results/aws_c5metal.json +++ b/website/benchmark/hardware/results/aws_c5metal_100.json @@ -1,6 +1,6 @@ [ { - "system": "AWS c5.metal", + "system": "AWS c5.metal 100GB", "system_full": "AWS c5.metal 96vCPU 192GiB 100GB SSD", "time": "2020-01-17 00:00:00", "kind": "cloud", diff --git a/website/benchmark/hardware/results/aws_c5metal_300.json b/website/benchmark/hardware/results/aws_c5metal_300.json new file mode 100644 index 00000000000..87435f6fb45 --- /dev/null +++ b/website/benchmark/hardware/results/aws_c5metal_300.json @@ -0,0 +1,54 @@ +[ + { + "system": "AWS c5.metal 300GB", + "system_full": "AWS c5.metal 96vCPU 192GiB 300GB SSD", + "time": "2020-09-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.012, 0.002, 0.002], +[0.066, 0.018, 0.018], +[0.066, 0.028, 0.027], +[0.186, 0.033, 0.031], +[0.362, 0.095, 0.093], +[1.092, 0.141, 0.142], +[0.035, 0.020, 0.021], +[0.023, 0.018, 0.018], +[0.303, 0.176, 0.181], +[0.817, 0.198, 0.198], +[0.322, 0.091, 0.092], +[0.600, 0.098, 0.098], +[1.059, 0.265, 0.253], +[1.542, 0.318, 0.310], +[0.682, 0.286, 0.283], +[0.372, 0.320, 0.295], +[1.610, 0.832, 0.750], +[1.301, 0.492, 0.458], +[3.446, 1.361, 1.330], +[0.189, 0.050, 0.035], +[9.246, 0.338, 0.265], +[10.163, 0.277, 0.249], +[19.616, 0.663, 0.639], +[20.068, 0.418, 0.367], +[1.812, 0.097, 0.093], +[0.976, 0.090, 0.083], +[2.458, 0.097, 0.095], +[9.397, 0.344, 0.323], +[7.320, 0.415, 0.413], +[0.780, 0.753, 0.748], +[1.328, 0.226, 0.223], +[4.643, 0.339, 0.329], +[4.136, 2.049, 2.021], +[9.213, 1.080, 0.923], +[9.192, 1.019, 0.959], +[0.410, 0.360, 0.378], +[0.244, 0.155, 0.163], +[0.102, 0.077, 0.071], +[0.045, 0.055, 0.049], +[0.459, 0.318, 0.316], +[0.069, 0.033, 0.026], +[0.035, 0.027, 0.020], +[0.019, 0.009, 0.010] + ] + } +] diff --git a/website/benchmark/hardware/results/aws_c6metal.json b/website/benchmark/hardware/results/aws_c6metal.json new file mode 100644 index 00000000000..83e75506ad9 --- /dev/null +++ b/website/benchmark/hardware/results/aws_c6metal.json @@ -0,0 +1,54 @@ +[ + { + "system": "AWS c6.metal (Graviton 2)", + "system_full": "AWS c6.metal (Graviton 2) 64 CPU 128GiB 2x1.7TB local SSD md-RAID-0", + "time": "2020-09-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.004, 0.003, 0.001], +[0.085, 0.030, 0.032], +[0.029, 0.028, 0.026], +[0.047, 0.068, 0.070], +[0.090, 0.075, 0.079], +[0.140, 0.126, 0.124], +[0.018, 0.013, 0.012], +[0.032, 0.021, 0.032], +[0.154, 0.139, 0.138], +[0.204, 0.155, 0.156], +[0.101, 0.091, 0.090], +[0.104, 0.104, 0.100], +[0.223, 0.203, 0.203], +[0.273, 0.255, 0.253], +[0.232, 0.212, 0.213], +[0.230, 0.223, 0.223], +[0.506, 0.484, 0.483], +[0.334, 0.330, 0.316], +[1.139, 1.085, 1.088], +[0.065, 0.077, 0.054], +[0.484, 0.315, 0.315], +[0.545, 0.295, 0.291], +[0.980, 0.661, 1.476], +[1.415, 1.101, 0.675], +[0.150, 0.086, 0.085], +[0.094, 0.077, 0.078], +[0.150, 0.087, 0.086], +[0.478, 0.348, 0.346], +[0.424, 0.403, 0.399], +[1.435, 1.388, 1.417], +[0.215, 0.178, 0.178], +[0.378, 0.294, 0.289], +[1.669, 1.590, 1.596], +[1.105, 1.007, 1.010], +[1.074, 1.041, 1.014], +[0.339, 0.323, 0.323], +[0.210, 0.199, 0.204], +[0.096, 0.091, 0.092], +[0.084, 0.080, 0.079], +[0.425, 0.405, 0.423], +[0.034, 0.025, 0.022], +[0.022, 0.019, 0.018], +[0.007, 0.007, 0.007] + ] + } +] diff --git a/website/benchmark/hardware/results/yandex_cloud_cascade_lake_32_vcpu.json b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_32_vcpu.json new file mode 100644 index 00000000000..5d2927c224d --- /dev/null +++ b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_32_vcpu.json @@ -0,0 +1,55 @@ +[ + { + "system": "Yandex Cloud 32vCPU", + "system_full": "Yandex Cloud Cascade Lake, 32 vCPU, 128 GB RAM, 300 GB SSD", + "cpu_vendor": "Intel", + "time": "2020-09-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.021, 0.001, 0.001], +[0.051, 0.011, 0.010], +[0.396, 0.025, 0.025], +[1.400, 0.035, 0.033], +[1.413, 0.095, 0.098], +[2.272, 0.222, 0.208], +[0.042, 0.014, 0.014], +[0.024, 0.011, 0.010], +[1.948, 0.311, 0.303], +[2.267, 0.379, 0.348], +[1.498, 0.138, 0.135], +[1.563, 0.164, 0.155], +[2.435, 0.544, 0.516], +[3.937, 0.661, 0.659], +[2.724, 0.727, 0.642], +[1.795, 0.683, 0.641], +[4.668, 1.682, 1.643], +[3.802, 1.051, 0.895], +[8.297, 3.835, 4.592], +[1.427, 0.100, 0.033], +[16.816, 0.652, 0.547], +[19.159, 0.650, 0.532], +[35.374, 1.538, 1.311], +[32.736, 0.854, 0.699], +[4.767, 0.203, 0.184], +[2.249, 0.166, 0.158], +[4.759, 0.207, 0.189], +[16.826, 0.584, 0.529], +[14.308, 0.920, 0.789], +[1.137, 1.041, 0.992], +[3.967, 0.545, 0.555], +[9.196, 0.872, 0.789], +[9.554, 5.501, 5.694], +[17.810, 2.712, 2.329], +[17.726, 2.653, 2.793], +[1.260, 0.955, 0.978], +[0.260, 0.171, 0.164], +[0.092, 0.065, 0.069], +[0.046, 0.041, 0.037], +[0.475, 0.391, 0.383], +[0.066, 0.021, 0.019], +[0.023, 0.024, 0.011], +[0.022, 0.005, 0.005] + ] + } +] diff --git a/website/benchmark/hardware/results/yandex_cloud_cascade_lake_80_vcpu.json b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_80_vcpu.json new file mode 100644 index 00000000000..565a5bd41c2 --- /dev/null +++ b/website/benchmark/hardware/results/yandex_cloud_cascade_lake_80_vcpu.json @@ -0,0 +1,55 @@ +[ + { + "system": "Yandex Cloud 80vCPU", + "system_full": "Yandex Cloud Cascade Lake, 80 vCPU, 160 GB RAM, 4TB SSD", + "cpu_vendor": "Intel", + "time": "2020-09-23 00:00:00", + "kind": "cloud", + "result": + [ +[0.024, 0.002, 0.002], +[0.067, 0.012, 0.012], +[0.104, 0.017, 0.017], +[0.411, 0.020, 0.021], +[0.577, 0.069, 0.068], +[0.739, 0.123, 0.122], +[0.038, 0.015, 0.014], +[0.024, 0.012, 0.012], +[0.625, 0.169, 0.168], +[0.748, 0.216, 0.207], +[0.471, 0.089, 0.082], +[0.487, 0.092, 0.087], +[0.818, 0.256, 0.245], +[1.324, 0.352, 0.352], +[0.927, 0.333, 0.319], +[0.642, 0.376, 0.377], +[1.686, 0.983, 0.959], +[1.290, 0.588, 0.582], +[3.105, 1.793, 1.818], +[0.426, 0.031, 0.034], +[5.559, 0.415, 0.344], +[6.343, 0.435, 0.405], +[11.779, 1.151, 1.101], +[11.851, 0.537, 0.509], +[1.530, 0.125, 0.126], +[0.695, 0.103, 0.103], +[1.531, 0.127, 0.119], +[5.576, 0.541, 0.496], +[4.718, 0.740, 0.719], +[1.429, 1.467, 1.500], +[1.309, 0.335, 0.322], +[3.138, 0.505, 0.518], +[5.481, 3.475, 3.512], +[6.330, 1.877, 1.818], +[6.238, 1.843, 1.813], +[0.660, 0.626, 0.603], +[0.251, 0.152, 0.151], +[0.090, 0.058, 0.059], +[0.041, 0.038, 0.034], +[0.470, 0.376, 0.385], +[0.076, 0.015, 0.018], +[0.030, 0.018, 0.010], +[0.024, 0.006, 0.005] + ] + } +] From 4e0378a521f6bb606723109b445e6ddd27bc48f8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 03:29:42 +0300 Subject: [PATCH 072/131] Adjustments --- benchmark/hardware.sh | 8 +++++--- website/benchmark/benchmark.js | 2 +- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 693a69deb1f..cfcbe6ce24c 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -60,7 +60,7 @@ if [[ ! -d data ]]; then if [[ ! -f $DATASET ]]; then $FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET" fi - + tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET fi @@ -105,8 +105,10 @@ echo echo "Benchmark complete. System info:" echo -echo '----Version and build id--------' -./clickhouse local --query "SELECT version(), buildId()" +echo '----Version, build id-----------' +./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" +./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" +./clickhouse local --query "SELECT format('Current time: {}', toString(now('UTC')))" echo '----CPU-------------------------' lscpu echo '----Block Devices---------------' diff --git a/website/benchmark/benchmark.js b/website/benchmark/benchmark.js index 6113864d4d1..8fb2693aa97 100644 --- a/website/benchmark/benchmark.js +++ b/website/benchmark/benchmark.js @@ -403,7 +403,7 @@ function generate_diagram() { var table_row = ""; table_row += ""; - table_row += " Date: Fri, 25 Sep 2020 03:36:06 +0300 Subject: [PATCH 073/131] Adjustments --- benchmark/hardware.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index cfcbe6ce24c..040f9169101 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -107,8 +107,8 @@ echo echo '----Version, build id-----------' ./clickhouse local --query "SELECT format('Version: {}, build id: {}', version(), buildId())" -./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" -./clickhouse local --query "SELECT format('Current time: {}', toString(now('UTC')))" +./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw +./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" echo '----CPU-------------------------' lscpu echo '----Block Devices---------------' From 092ea18dd4d81fd4e3b89b748c85c33092e87f52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 03:48:18 +0300 Subject: [PATCH 074/131] Adjustments --- website/benchmark/hardware/results/aws_m6g_16xlarge.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/benchmark/hardware/results/aws_m6g_16xlarge.json b/website/benchmark/hardware/results/aws_m6g_16xlarge.json index 323fd2cc50c..a0d15a0d384 100644 --- a/website/benchmark/hardware/results/aws_m6g_16xlarge.json +++ b/website/benchmark/hardware/results/aws_m6g_16xlarge.json @@ -1,7 +1,7 @@ [ { - "system": "AWS m6g.16xlarge", - "system_full": "AWS m6g.16xlarge (Graviton2) 64 vCPU, 256 GiB RAM, EBS", + "system": "AWS m6g.16xlarge (Graviton 2)", + "system_full": "AWS m6g.16xlarge (Graviton 2) 64 vCPU, 256 GiB RAM, EBS", "time": "2020-02-13 00:00:00", "kind": "cloud", "result": From 1bd6fb39f217a6464476c2174abeae416960d2f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 03:55:57 +0300 Subject: [PATCH 075/131] Added a comment --- benchmark/hardware.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 040f9169101..90e0e7dadfa 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -14,6 +14,8 @@ TRIES=3 AMD64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_build_check/gcc-10_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" AARCH64_BIN_URL="https://clickhouse-builds.s3.yandex.net/0/e29c4c3cc47ab2a6c4516486c1b77d57e7d42643/clickhouse_special_build_check/clang-10-aarch64_relwithdebuginfo_none_bundled_unsplitted_disable_False_binary/clickhouse" +# Note: on older Ubuntu versions, 'axel' does not support IPv6. If you are using IPv6-only servers on very old Ubuntu, just don't install 'axel'. + FASTER_DOWNLOAD=wget if command -v axel >/dev/null; then FASTER_DOWNLOAD=axel From fdf860adde2dc20fcedc1ffc0f78ed9d82f8ee57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 04:03:17 +0300 Subject: [PATCH 076/131] More handy --- benchmark/hardware.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 90e0e7dadfa..bb3492844a7 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -66,6 +66,8 @@ if [[ ! -d data ]]; then tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET fi +uptime + echo "Starting clickhouse-server" ./clickhouse server > server.log 2>&1 & From 8b55224350ea0c93fcfb2d0fd9830a00dfba6a66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 04:06:21 +0300 Subject: [PATCH 077/131] More handy --- benchmark/hardware.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index bb3492844a7..286482d9132 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -114,6 +114,7 @@ echo '----Version, build id-----------' ./clickhouse local --query "SELECT format('The number of threads is: {}', value) FROM system.settings WHERE name = 'max_threads'" --output-format TSVRaw ./clickhouse local --query "SELECT format('Current time: {}', toString(now(), 'UTC'))" echo '----CPU-------------------------' +cat /proc/cpuinfo | grep -i -F 'model name' | uniq lscpu echo '----Block Devices---------------' lsblk From 0e7f576bc1256faf86bddf97c36b465a5e738a8f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 04:21:03 +0300 Subject: [PATCH 078/131] More results --- .../hardware/results/xeon_e5_2650_4hdd.json | 54 +++++++++++++++++++ ...on_e5_2650.json => xeon_e5_2650_8hdd.json} | 0 2 files changed, 54 insertions(+) create mode 100644 website/benchmark/hardware/results/xeon_e5_2650_4hdd.json rename website/benchmark/hardware/results/{xeon_e5_2650.json => xeon_e5_2650_8hdd.json} (100%) diff --git a/website/benchmark/hardware/results/xeon_e5_2650_4hdd.json b/website/benchmark/hardware/results/xeon_e5_2650_4hdd.json new file mode 100644 index 00000000000..478229badcc --- /dev/null +++ b/website/benchmark/hardware/results/xeon_e5_2650_4hdd.json @@ -0,0 +1,54 @@ +[ + { + "system": "Xeon E5-2650", + "system_full": "Xeon E5-2650 v2 @ 2.60GHz, 2 sockets, 16 threads, 4xHDD RAID-10", + "time": "2020-09-25 00:00:00", + "kind": "server", + "result": + [ +[0.040, 0.002, 0.002], +[0.698, 0.014, 0.013], +[0.533, 0.030, 0.030], +[0.700, 0.043, 0.046], +[0.749, 0.108, 0.102], +[1.350, 0.221, 0.259], +[0.168, 0.020, 0.020], +[0.096, 0.013, 0.013], +[1.132, 0.406, 0.386], +[1.279, 0.426, 0.440], +[0.842, 0.153, 0.146], +[1.042, 0.186, 0.182], +[1.149, 0.536, 0.533], +[1.734, 0.688, 0.683], +[1.481, 0.688, 0.651], +[1.100, 0.709, 0.700], +[2.367, 1.668, 1.682], +[1.687, 1.013, 0.988], +[4.768, 3.647, 3.783], +[0.599, 0.055, 0.040], +[5.530, 0.646, 0.622], +[6.658, 0.671, 0.648], +[11.795, 1.645, 1.574], +[19.248, 1.168, 0.906], +[1.826, 0.224, 0.232], +[0.964, 0.189, 0.187], +[2.058, 0.234, 0.215], +[5.811, 0.758, 0.704], +[4.805, 1.014, 0.995], +[2.272, 2.035, 1.838], +[1.827, 0.546, 0.547], +[3.643, 0.863, 0.834], +[5.816, 5.069, 5.168], +[6.585, 2.655, 2.756], +[6.949, 2.681, 2.795], +[1.325, 1.090, 1.072], +[0.460, 0.183, 0.179], +[1.000, 0.087, 0.091], +[0.142, 0.051, 0.038], +[0.808, 0.392, 0.391], +[0.256, 0.021, 0.015], +[0.132, 0.038, 0.012], +[0.054, 0.006, 0.006] + ] + } +] diff --git a/website/benchmark/hardware/results/xeon_e5_2650.json b/website/benchmark/hardware/results/xeon_e5_2650_8hdd.json similarity index 100% rename from website/benchmark/hardware/results/xeon_e5_2650.json rename to website/benchmark/hardware/results/xeon_e5_2650_8hdd.json From f5eead5b2aba0e4c719440c03b73bf0bc5caa32a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 04:45:47 +0300 Subject: [PATCH 079/131] Enable embedded configs for AArch64 --- programs/server/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/server/CMakeLists.txt b/programs/server/CMakeLists.txt index 5500a4680b7..b3dcf1955fe 100644 --- a/programs/server/CMakeLists.txt +++ b/programs/server/CMakeLists.txt @@ -3,7 +3,7 @@ set(CLICKHOUSE_SERVER_SOURCES Server.cpp ) -if (OS_LINUX AND ARCH_AMD64) +if (OS_LINUX) set (LINK_CONFIG_LIB INTERFACE "-Wl,${WHOLE_ARCHIVE} $ -Wl,${NO_WHOLE_ARCHIVE}") endif () From 4e9d37e033ca0964b64b89aa8d4607685fcaaf3a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 04:46:13 +0300 Subject: [PATCH 080/131] More convenient --- benchmark/hardware.sh | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/benchmark/hardware.sh b/benchmark/hardware.sh index 693a69deb1f..bd932a57cda 100755 --- a/benchmark/hardware.sh +++ b/benchmark/hardware.sh @@ -36,14 +36,6 @@ if [[ ! -f clickhouse ]]; then $FASTER_DOWNLOAD "$AMD64_BIN_URL" elif [[ $CPU == aarch64 ]]; then $FASTER_DOWNLOAD "$AARCH64_BIN_URL" - - # Download configs. ARM version has no embedded configs. - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.xml - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/users.xml - mkdir config.d - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/path.xml -O config.d/path.xml - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/access_control.xml -O config.d/access_control.xml - wget https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/programs/server/config.d/log_to_console.xml -O config.d/log_to_console.xml else echo "Unsupported CPU type: $CPU" exit 1 @@ -60,7 +52,7 @@ if [[ ! -d data ]]; then if [[ ! -f $DATASET ]]; then $FASTER_DOWNLOAD "https://clickhouse-datasets.s3.yandex.net/hits/partitions/$DATASET" fi - + tar $TAR_PARAMS --strip-components=1 --directory=. -x -v -f $DATASET fi From 23fd72a3deb31a44272386c4a4399a05c58151be Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 25 Sep 2020 04:59:01 +0300 Subject: [PATCH 081/131] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 6d57dfde9cd..0cbfdcb7d81 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -38,7 +38,7 @@ toc_title: Adopters | Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | | Diva-e | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | | Ecwid | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) | -| eBay | E-commerce | TBA | — | — | [Webinar, Sep 2020](https://altinity.com/webinarspage/2020/09/08/migrating-from-druid-to-next-gen-olap-on-clickhouse-ebays-experience) | +| eBay | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) | | Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | | FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | | Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | From 1f6e55ff35615351318c661ef4c9a24c69498b7a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 05:03:58 +0300 Subject: [PATCH 082/131] Allow to run without /proc/self/maps --- programs/server/Server.cpp | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 6d47e7e0364..b703d0ac6a7 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -339,16 +339,23 @@ int Server::main(const std::vector & /*args*/) { if (hasLinuxCapability(CAP_IPC_LOCK)) { - /// Get the memory area with (current) code segment. - /// It's better to lock only the code segment instead of calling "mlockall", - /// because otherwise debug info will be also locked in memory, and it can be huge. - auto [addr, len] = getMappedArea(reinterpret_cast(mainEntryClickHouseServer)); + try + { + /// Get the memory area with (current) code segment. + /// It's better to lock only the code segment instead of calling "mlockall", + /// because otherwise debug info will be also locked in memory, and it can be huge. + auto [addr, len] = getMappedArea(reinterpret_cast(mainEntryClickHouseServer)); - LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds."); - if (0 != mlock(addr, len)) - LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR)); - else - LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len)); + LOG_TRACE(log, "Will do mlock to prevent executable memory from being paged out. It may take a few seconds."); + if (0 != mlock(addr, len)) + LOG_WARNING(log, "Failed mlock: {}", errnoToString(ErrorCodes::SYSTEM_ERROR)); + else + LOG_TRACE(log, "The memory map of clickhouse executable has been mlock'ed, total {}", ReadableSize(len)); + } + catch (...) + { + LOG_WARNING(log, "Cannot mlock: {}", getCurrentExceptionMessage(false)); + } } else { From 85bfdbad24760fb507dd6497c50a132073853af8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 05:12:51 +0300 Subject: [PATCH 083/131] Add a test for embedded configs --- ...erver_start_with_embedded_config.reference | 5 ++ ...house_server_start_with_embedded_config.sh | 48 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.reference create mode 100755 tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh diff --git a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.reference b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.reference new file mode 100644 index 00000000000..c3829d603de --- /dev/null +++ b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.reference @@ -0,0 +1,5 @@ +Starting clickhouse-server +Waiting for clickhouse-server to start +1 +Hello +World diff --git a/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh new file mode 100755 index 00000000000..68198ec6e16 --- /dev/null +++ b/tests/queries/0_stateless/01507_clickhouse_server_start_with_embedded_config.sh @@ -0,0 +1,48 @@ +#!/usr/bin/env bash + +CLICKHOUSE_PORT_TCP=50111 +CLICKHOUSE_DATABASE=default + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +echo "Starting clickhouse-server" + +$PORT + +$CLICKHOUSE_BINARY server -- --tcp_port "$CLICKHOUSE_PORT_TCP" > server.log 2>&1 & +PID=$! + +function finish { + kill $PID + wait +} +trap finish EXIT + +echo "Waiting for clickhouse-server to start" + +for i in {1..30}; do + sleep 1 + $CLICKHOUSE_CLIENT --query "SELECT 1" 2>/dev/null && break + if [[ $i == 30 ]]; then + cat server.log + exit 1 + fi +done + +# Check access rights + +$CLICKHOUSE_CLIENT -n --query " + DROP DATABASE IF EXISTS test; + CREATE DATABASE test; + USE test; + + CREATE TABLE t (s String) ENGINE=TinyLog; + INSERT INTO t VALUES ('Hello'); + SELECT * FROM t; + DROP TABLE t; + + CREATE TEMPORARY TABLE t (s String); + INSERT INTO t VALUES ('World'); + SELECT * FROM t; +"; From 161ee1df3ed262191f6c10eff9046f4ab62db8e6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 25 Sep 2020 05:27:13 +0300 Subject: [PATCH 084/131] Fail more quickly in clickhouse-test if server does not respond --- tests/clickhouse-test | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a3bed189d55..3f0c4bd155a 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -303,6 +303,14 @@ def run_tests_array(all_tests_with_params): clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) clickhouse_proc.communicate("SELECT 'Running test {suite}/{case} from pid={pid}';".format(pid = os.getpid(), case = case, suite = suite)) + if clickhouse_proc.returncode != 0: + failures += 1 + print(MSG_FAIL, end='') + print_test_time(0) + print(" - server does not respond to health check") + SERVER_DIED = True + break + reference_file = os.path.join(suite_dir, name) + '.reference' stdout_file = os.path.join(suite_tmp_dir, name) + '.stdout' stderr_file = os.path.join(suite_tmp_dir, name) + '.stderr' From 23f06de0bef472ccf18b950c2afc7629c7266a65 Mon Sep 17 00:00:00 2001 From: John <108540613@qq.com> Date: Fri, 25 Sep 2020 11:48:25 +0800 Subject: [PATCH 085/131] Update in.md --- docs/zh/sql-reference/operators/in.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/operators/in.md b/docs/zh/sql-reference/operators/in.md index eaaa477fbe1..bcd3ca1fa18 100644 --- a/docs/zh/sql-reference/operators/in.md +++ b/docs/zh/sql-reference/operators/in.md @@ -3,7 +3,7 @@ machine_translated: true machine_translated_rev: 5decc73b5dc60054f19087d3690c4eb99446a6c3 --- -# 在运营商 {#select-in-operators} +# IN 操作符 {#select-in-operators} 该 `IN`, `NOT IN`, `GLOBAL IN`,和 `GLOBAL NOT IN` 运算符是单独复盖的,因为它们的功能相当丰富。 From 049f21091e35f5dbff1182128bcafe154700e120 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 25 Sep 2020 08:05:50 +0300 Subject: [PATCH 086/131] Update clickhouse-test --- tests/clickhouse-test | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 3f0c4bd155a..36467f18d55 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -305,9 +305,7 @@ def run_tests_array(all_tests_with_params): if clickhouse_proc.returncode != 0: failures += 1 - print(MSG_FAIL, end='') - print_test_time(0) - print(" - server does not respond to health check") + print("Server does not respond to health check") SERVER_DIED = True break From 6d865449117afead01e3bce527577487f943d5d9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 10:55:41 +0300 Subject: [PATCH 087/131] Trying to supress race in LazyPipeFDs --- tests/tsan_suppressions.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index 912e0361bff..b12001f4f42 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1 +1,4 @@ # Fortunately, we have no suppressions! + +# looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method +# race:DB::LazyPipeFDs::close From 61ab1a264dedcb84654efb9f2f69f977d712c1b8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 10:56:30 +0300 Subject: [PATCH 088/131] Suppress race --- tests/tsan_suppressions.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index b12001f4f42..35c4ba856e6 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1,4 +1,4 @@ # Fortunately, we have no suppressions! # looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method -# race:DB::LazyPipeFDs::close +race:DB::LazyPipeFDs::close From 81d08b59e51b0420e51b0ffd7750c8bfa34d4c93 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 25 Sep 2020 16:38:09 +0800 Subject: [PATCH 089/131] Replace useless multiset with unordered_set --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/VirtualColumnUtils.h | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2b8b886daaf..8b5b337bcec 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -223,7 +223,7 @@ Pipe MergeTreeDataSelectExecutor::readFromParts( if (part_column_queried) VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context); - std::multiset part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + auto part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 89b69eb79e3..445a996ab87 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -30,9 +30,9 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c /// Extract from the input stream a set of `name` column values template -std::multiset extractSingleValueFromBlock(const Block & block, const String & name) +auto extractSingleValueFromBlock(const Block & block, const String & name) { - std::multiset res; + std::unordered_set res; const ColumnWithTypeAndName & data = block.getByName(name); size_t rows = block.rows(); for (size_t i = 0; i < rows; ++i) From 4666ae3211667431d73d0b474693716bec3b12dd Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 25 Sep 2020 17:59:19 +0800 Subject: [PATCH 090/131] fix build bug for merge_selector2.cpp --- src/Storages/MergeTree/MergeSelector.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeSelector.h b/src/Storages/MergeTree/MergeSelector.h index e460b8ae06a..fcdfcf5b890 100644 --- a/src/Storages/MergeTree/MergeSelector.h +++ b/src/Storages/MergeTree/MergeSelector.h @@ -44,7 +44,7 @@ public: /// Information about different TTLs for part. Can be used by /// TTLSelector to assign merges with TTL. - const MergeTreeDataPartTTLInfos * ttl_infos; + const MergeTreeDataPartTTLInfos * ttl_infos = nullptr; /// Part compression codec definition. ASTPtr compression_codec_desc; From 3f717ea1c471f9c15e617003c9a52ae6776552ba Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 25 Sep 2020 13:39:59 +0300 Subject: [PATCH 091/131] fix skip lists --- tests/queries/0_stateless/arcadia_skip_list.txt | 4 ++++ tests/queries/skip_list.json | 9 ++++----- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 6d1c6444d1b..3191eb87fed 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -145,4 +145,8 @@ 01461_query_start_time_microseconds 01455_shard_leaf_max_rows_bytes_to_read 01505_distributed_local_type_conversion_enum +00604_show_create_database +00609_mv_index_in_in +00510_materizlized_view_and_deduplication_zookeeper +00738_lock_for_inner_table 01505_log_distributed_deadlock diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index abd6ed83304..26e5bbf78cf 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -89,10 +89,9 @@ "release-build": [ ], "database-ordinary": [ - "00604_show_create_database" - ], - "polymorphic-parts": [ - "00933_test_fix_extra_seek_on_compressed_cache", - "00446_clear_column_in_partition_zookeeper" + "00604_show_create_database", + "00609_mv_index_in_in", + "00510_materizlized_view_and_deduplication_zookeeper", + "00738_lock_for_inner_table" ] } From d3f3867445ed9aaf459b9931176f8e311baeacf4 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 13:46:38 +0300 Subject: [PATCH 092/131] Update tsan_suppressions.txt --- tests/tsan_suppressions.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/tsan_suppressions.txt b/tests/tsan_suppressions.txt index 35c4ba856e6..668710a33d7 100644 --- a/tests/tsan_suppressions.txt +++ b/tests/tsan_suppressions.txt @@ -1,4 +1,2 @@ -# Fortunately, we have no suppressions! - # looks like a bug in clang-11 thread sanitizer, detects normal data race with random FD in this method race:DB::LazyPipeFDs::close From 97671a17719eb9c101095e01cac7baa282908126 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 14:27:00 +0300 Subject: [PATCH 093/131] Ugly fix for default database race --- programs/server/Server.cpp | 13 +++++++++---- src/Interpreters/Context.cpp | 12 ++++++++++++ src/Interpreters/Context.h | 3 +++ src/Interpreters/SystemLog.h | 2 +- 4 files changed, 25 insertions(+), 5 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b703d0ac6a7..6341653ee2f 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -671,6 +671,10 @@ int Server::main(const std::vector & /*args*/) total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); + /// Set current database name before loading tables and databases because + /// system logs may copy global context. + global_context->setCurrentDatabaseNameInGlobalContext(default_database); + LOG_INFO(log, "Loading metadata from {}", path); try @@ -678,11 +682,14 @@ int Server::main(const std::vector & /*args*/) loadMetadataSystem(*global_context); /// After attaching system databases we can initialize system log. global_context->initializeSystemLogs(); + auto & database_catalog = DatabaseCatalog::instance(); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) - attachSystemTablesServer(*DatabaseCatalog::instance().getSystemDatabase(), has_zookeeper); + attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper); /// Then, load remaining databases loadMetadata(*global_context, default_database); - DatabaseCatalog::instance().loadDatabases(); + database_catalog.loadDatabases(); + /// After loading validate that default database exists + database_catalog.assertDatabaseExists(default_database); } catch (...) { @@ -745,8 +752,6 @@ int Server::main(const std::vector & /*args*/) LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created" " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe)."); - global_context->setCurrentDatabase(default_database); - if (has_zookeeper && config().has("distributed_ddl")) { /// DDL worker should be started after all tables were loaded diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 704b21f3a4a..be35c8a9184 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1088,6 +1088,18 @@ String Context::getInitialQueryId() const } +void Context::setCurrentDatabaseNameInGlobalContext(const String & name) +{ + if (global_context != this) + throw Exception("Cannot set current database for non global context, this method should be used during server initialization", ErrorCodes::LOGICAL_ERROR); + auto lock = getLock(); + + if (!current_database.empty()) + throw Exception("Default database name cannot be changed in global context without server restart", ErrorCodes::LOGICAL_ERROR); + + current_database = name; +} + void Context::setCurrentDatabase(const String & name) { DatabaseCatalog::instance().assertDatabaseExists(name); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 3d66ef239e7..bd5e17fe2e4 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -359,6 +359,9 @@ public: String getInitialQueryId() const; void setCurrentDatabase(const String & name); + /// Set current_database for global context. We don't validate that database + /// exists because it should be set before databases loading. + void setCurrentDatabaseNameInGlobalContext(const String & name); void setCurrentQueryId(const String & query_id); void killCurrentQuery(); diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 03b1b735cbc..2a0ce9cef53 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -438,7 +438,7 @@ void SystemLog::flushImpl(const std::vector & to_flush, ASTPtr query_ptr(insert.release()); // we need query context to do inserts to target table with MV containing subqueries or joins - auto insert_context = Context(context); + Context insert_context(context); insert_context.makeQueryContext(); InterpreterInsertQuery interpreter(query_ptr, insert_context); From b30e86016635da8bc0197ee3b93694c431ba33e5 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 25 Sep 2020 14:51:00 +0300 Subject: [PATCH 094/131] Update ASTColumnsTransformers.h --- src/Parsers/ASTColumnsTransformers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index ddf0d70dc35..4b7a933647e 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -53,7 +53,7 @@ public: ASTPtr clone() const override { auto replacement = std::make_shared(*this); - replacement->name = name; + replacement->children.clear(); replacement->expr = expr->clone(); replacement->children.push_back(replacement->expr); return replacement; From 8f21c73b763ce86d2032648d00325baf9945d116 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 25 Sep 2020 14:52:55 +0300 Subject: [PATCH 095/131] Revert "Update ASTColumnsTransformers.h" This reverts commit b30e86016635da8bc0197ee3b93694c431ba33e5. --- src/Parsers/ASTColumnsTransformers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index 4b7a933647e..ddf0d70dc35 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -53,7 +53,7 @@ public: ASTPtr clone() const override { auto replacement = std::make_shared(*this); - replacement->children.clear(); + replacement->name = name; replacement->expr = expr->clone(); replacement->children.push_back(replacement->expr); return replacement; From d4c8ba84102c948f1da37c3f00abd1ca6fa2a15f Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 25 Sep 2020 14:58:04 +0300 Subject: [PATCH 096/131] Update ASTColumnsTransformers.h --- src/Parsers/ASTColumnsTransformers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnsTransformers.h b/src/Parsers/ASTColumnsTransformers.h index ddf0d70dc35..4b7a933647e 100644 --- a/src/Parsers/ASTColumnsTransformers.h +++ b/src/Parsers/ASTColumnsTransformers.h @@ -53,7 +53,7 @@ public: ASTPtr clone() const override { auto replacement = std::make_shared(*this); - replacement->name = name; + replacement->children.clear(); replacement->expr = expr->clone(); replacement->children.push_back(replacement->expr); return replacement; From f5c38b34ae5f439a46c661413a429efc30aca05f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 15:07:38 +0300 Subject: [PATCH 097/131] Fix race condition during write ahead log cleanup and rename --- src/Storages/MergeTree/MergeTreeData.h | 2 ++ src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp | 4 +++- src/Storages/StorageMergeTree.cpp | 4 +++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 0fc5ec43048..1125eb32b66 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -717,6 +717,8 @@ protected: bool require_part_metadata; + /// Relative path data, changes during rename for ordinary databases use + /// under lockForShare if rename is possible. String relative_data_path; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index a5216e6fda3..11f23a5c110 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -56,10 +56,12 @@ void ReplicatedMergeTreeCleanupThread::run() void ReplicatedMergeTreeCleanupThread::iterate() { storage.clearOldPartsAndRemoveFromZK(); - storage.clearOldWriteAheadLogs(); { auto lock = storage.lockForShare(RWLockImpl::NO_QUERY, storage.getSettings()->lock_acquire_timeout_for_background_operations); + /// Both use relative_data_path which changes during rename, so we + /// do it under share lock + storage.clearOldWriteAheadLogs(); storage.clearOldTemporaryDirectories(); } diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 347474753dc..55fb42b550e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -919,11 +919,13 @@ BackgroundProcessingPoolTaskResult StorageMergeTree::mergeMutateTask() { { auto share_lock = lockForShare(RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); + /// All use relative_data_path which changes during rename + /// so execute under share lock. clearOldPartsFromFilesystem(); clearOldTemporaryDirectories(); + clearOldWriteAheadLogs(); } clearOldMutations(); - clearOldWriteAheadLogs(); } ///TODO: read deduplicate option from table config From 86322fa3e1b86a2e96452805c18e440c29e6dab0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 15:36:02 +0300 Subject: [PATCH 098/131] Add test --- ...condition_rename_clear_zookeeper.reference | 1 + ...8_race_condition_rename_clear_zookeeper.sh | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference create mode 100755 tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh diff --git a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference new file mode 100644 index 00000000000..13de30f45d1 --- /dev/null +++ b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.reference @@ -0,0 +1 @@ +3000 diff --git a/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh new file mode 100755 index 00000000000..2af1cb214a4 --- /dev/null +++ b/tests/queries/0_stateless/01508_race_condition_rename_clear_zookeeper.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50" + + +$CLICKHOUSE_CLIENT --query "CREATE TABLE table_for_renames0 (value UInt64, data String) ENGINE ReplicatedMergeTree('/clickhouse/tables/test_01508/concurrent_rename', '1') ORDER BY tuple() SETTINGS cleanup_delay_period = 1, cleanup_delay_period_random_add = 0, min_rows_for_compact_part = 100000, min_rows_for_compact_part = 10000000, write_ahead_log_max_bytes = 1" + + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(1000, 1000)" + +$CLICKHOUSE_CLIENT --query "INSERT INTO table_for_renames0 SELECT number, toString(number) FROM numbers(2000, 1000)" + +for i in $(seq 1 50); do + prev_i=$((i - 1)) + $CLICKHOUSE_CLIENT --query "RENAME TABLE table_for_renames$prev_i TO table_for_renames$i" +done + +$CLICKHOUSE_CLIENT --query "SELECT COUNT() from table_for_renames50" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames0" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_for_renames50" From 39c356f23061d08e9f662fd254c01a385e6facef Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 25 Sep 2020 16:12:52 +0300 Subject: [PATCH 099/131] Change phone number --- docs/en/introduction/info.md | 10 ++++++++++ docs/ru/introduction/info.md | 2 +- 2 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 docs/en/introduction/info.md diff --git a/docs/en/introduction/info.md b/docs/en/introduction/info.md new file mode 100644 index 00000000000..a397c40950d --- /dev/null +++ b/docs/en/introduction/info.md @@ -0,0 +1,10 @@ +--- +toc_priority: 100 +--- + +# Information support {#information-support} + +- Email address: +- Phone: +7-495-780-6510 + +[Original article](https://clickhouse.tech/docs/en/introduction/info/) \ No newline at end of file diff --git a/docs/ru/introduction/info.md b/docs/ru/introduction/info.md index 14e517eebae..a9398b8c9cd 100644 --- a/docs/ru/introduction/info.md +++ b/docs/ru/introduction/info.md @@ -7,6 +7,6 @@ toc_priority: 100 Информационная поддержка ClickHouse осуществляется на всей территории Российской Федерации без ограничений посредством использования телефонной связи и средств электронной почты на русском языке в круглосуточном режиме: - Адрес электронной почты: -- Телефон: 8-800-250-96-39 (звонки бесплатны из всех регионов России) +- Телефон: +7-495-780-6510 [Оригинальная статья](https://clickhouse.tech/docs/ru/introduction/info/) From 54fa67838e95364c5169ccb67b8e7dfc0b90e337 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 16:32:00 +0300 Subject: [PATCH 100/131] Add flaky test --- tests/integration/test_flaky_check/__init__.py | 0 tests/integration/test_flaky_check/test.py | 5 +++++ 2 files changed, 5 insertions(+) create mode 100644 tests/integration/test_flaky_check/__init__.py create mode 100644 tests/integration/test_flaky_check/test.py diff --git a/tests/integration/test_flaky_check/__init__.py b/tests/integration/test_flaky_check/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_flaky_check/test.py b/tests/integration/test_flaky_check/test.py new file mode 100644 index 00000000000..4da85151c46 --- /dev/null +++ b/tests/integration/test_flaky_check/test.py @@ -0,0 +1,5 @@ +import random + + +def test_flaky(): + assert random.randint(0, 2) >= 1 From ada193e7b716f1ebd04abefe8fbbdcb225e1f766 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 16:33:33 +0300 Subject: [PATCH 101/131] Add diff --- tests/integration/test_ttl_move/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 377ee0e5d75..a4f8606011d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -8,6 +8,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +# test cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', From 6914a8667311b8f784c36b9a77c15580b469844c Mon Sep 17 00:00:00 2001 From: Maxim Akhmedov Date: Fri, 25 Sep 2020 10:31:06 +0300 Subject: [PATCH 102/131] Put createBlockForSet declaration to header file. --- src/Interpreters/ActionsVisitor.cpp | 15 ++------------- src/Interpreters/ActionsVisitor.h | 28 +++++++++++++++++++++++++++- 2 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index be040ff2c34..1d524669fd9 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -235,11 +235,7 @@ static Block createBlockFromAST(const ASTPtr & node, const DataTypes & types, co return header.cloneWithColumns(std::move(columns)); } -/** Create a block for set from literal. - * 'set_element_types' - types of what are on the left hand side of IN. - * 'right_arg' - Literal - Tuple or Array. - */ -static Block createBlockForSet( +Block createBlockForSet( const DataTypePtr & left_arg_type, const ASTPtr & right_arg, const DataTypes & set_element_types, @@ -280,14 +276,7 @@ static Block createBlockForSet( return block; } -/** Create a block for set from expression. - * 'set_element_types' - types of what are on the left hand side of IN. - * 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6). - * - * We need special implementation for ASTFunction, because in case, when we interpret - * large tuple or array as function, `evaluateConstantExpression` works extremely slow. - */ -static Block createBlockForSet( +Block createBlockForSet( const DataTypePtr & left_arg_type, const std::shared_ptr & right_arg, const DataTypes & set_element_types, diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index d8d85f1c0bf..98ea3f79fff 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -16,11 +16,37 @@ struct ExpressionAction; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; - /// The case of an explicit enumeration of values. +/// The case of an explicit enumeration of values. SetPtr makeExplicitSet( const ASTFunction * node, const Block & sample_block, bool create_ordered_set, const Context & context, const SizeLimits & limits, PreparedSets & prepared_sets); +/** Create a block for set from expression. + * 'set_element_types' - types of what are on the left hand side of IN. + * 'right_arg' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6). + * + * We need special implementation for ASTFunction, because in case, when we interpret + * large tuple or array as function, `evaluateConstantExpression` works extremely slow. + * + * Note: this and following functions are used in third-party applications in Arcadia, so + * they should be declared in header file. + * + */ +Block createBlockForSet( + const DataTypePtr & left_arg_type, + const std::shared_ptr & right_arg, + const DataTypes & set_element_types, + const Context & context); + +/** Create a block for set from literal. + * 'set_element_types' - types of what are on the left hand side of IN. + * 'right_arg' - Literal - Tuple or Array. + */ +Block createBlockForSet( + const DataTypePtr & left_arg_type, + const ASTPtr & right_arg, + const DataTypes & set_element_types, + const Context & context); /** For ActionsVisitor * A stack of ExpressionActions corresponding to nested lambda expressions. From af0bb5bd263d0b12600f0ea2d86b0cf6a6d76bf1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 19:15:34 +0300 Subject: [PATCH 103/131] One more time --- tests/integration/test_s3_with_proxy/test.py | 14 ++++++++++---- tests/integration/test_ttl_move/test.py | 1 - 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 9df209826f9..3d118266455 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -1,5 +1,6 @@ import logging import os +import time import pytest from helpers.cluster import ClickHouseCluster @@ -37,10 +38,15 @@ def cluster(): def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): - logs = cluster.get_container_logs(proxy_instance) - # Check that all possible interactions with Minio are present - for http_method in http_methods: - assert logs.find(http_method + " http://minio1") >= 0 + for i in range(10): + logs = cluster.get_container_logs(proxy_instance) + # Check that all possible interactions with Minio are present + for http_method in http_methods: + if logs.find(http_method + " http://minio1") >= 0: + return + time.sleep(1) + else: + assert False, "http method not found in logs" @pytest.mark.parametrize( diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index a4f8606011d..377ee0e5d75 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -8,7 +8,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -# test cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', From 8f9ec303154fc3e04222a4c999ec6f8e8a1bcf01 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 19:23:57 +0300 Subject: [PATCH 104/131] Delete flaky test --- tests/integration/test_flaky_check/__init__.py | 0 tests/integration/test_flaky_check/test.py | 5 ----- 2 files changed, 5 deletions(-) delete mode 100644 tests/integration/test_flaky_check/__init__.py delete mode 100644 tests/integration/test_flaky_check/test.py diff --git a/tests/integration/test_flaky_check/__init__.py b/tests/integration/test_flaky_check/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_flaky_check/test.py b/tests/integration/test_flaky_check/test.py deleted file mode 100644 index 4da85151c46..00000000000 --- a/tests/integration/test_flaky_check/test.py +++ /dev/null @@ -1,5 +0,0 @@ -import random - - -def test_flaky(): - assert random.randint(0, 2) >= 1 From 5d8de1f65d8b3728847ae6a5ae5aca8f6f070c2e Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 25 Sep 2020 20:31:01 +0300 Subject: [PATCH 105/131] Update ci_config.json --- tests/ci/ci_config.json | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 220d8d801ec..3d41e8495e9 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -237,7 +237,7 @@ "with_coverage": false } }, - "Functional stateful tests (release, DatabaseAtomic)": { + "Functional stateful tests (release, DatabaseOrdinary)": { "required_build_properties": { "compiler": "gcc-10", "package_type": "deb", @@ -345,7 +345,7 @@ "with_coverage": false } }, - "Functional stateless tests (release, DatabaseAtomic)": { + "Functional stateless tests (release, DatabaseOrdinary)": { "required_build_properties": { "compiler": "gcc-10", "package_type": "deb", From 35ab3657c3cd2e313279733f538f43fccbb903ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 25 Sep 2020 21:39:55 +0300 Subject: [PATCH 106/131] Add flaky tests config --- tests/ci/ci_config.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/ci/ci_config.json b/tests/ci/ci_config.json index 220d8d801ec..c01025ea865 100644 --- a/tests/ci/ci_config.json +++ b/tests/ci/ci_config.json @@ -441,6 +441,18 @@ "with_coverage": false } }, + "Integration tests flaky check (asan)": { + "required_build_properties": { + "compiler": "clang-11", + "package_type": "deb", + "build_type": "relwithdebuginfo", + "sanitizer": "address", + "bundled": "bundled", + "splitted": "unsplitted", + "clang-tidy": "disable", + "with_coverage": false + } + }, "Compatibility check": { "required_build_properties": { "compiler": "gcc-10", From 62c2c2000cd0c33d8a8b39d7c34a0104848af9b7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:00:54 +0300 Subject: [PATCH 107/131] Query obfuscator: development --- src/CMakeLists.txt | 2 +- src/Common/BitHelpers.h | 43 +- src/Common/StringUtils/StringUtils.h | 23 +- src/Parsers/obfuscateQueries.cpp | 934 ++++++++++++++++++ src/Parsers/obfuscateQueries.h | 29 + src/Parsers/tests/gtest_obfuscate_queries.cpp | 92 ++ 6 files changed, 1104 insertions(+), 19 deletions(-) create mode 100644 src/Parsers/obfuscateQueries.cpp create mode 100644 src/Parsers/obfuscateQueries.h create mode 100644 src/Parsers/tests/gtest_obfuscate_queries.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0016c51b7f8..40815228eac 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -415,6 +415,6 @@ if (ENABLE_TESTS AND USE_GTEST) -Wno-gnu-zero-variadic-macro-arguments ) - target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) + target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) add_check(unit_tests_dbms) endif () diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index 699e379b8d3..eac5fdac80e 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -1,22 +1,12 @@ #pragma once #include +#include #include #include #include -/** Returns log2 of number, rounded down. - * Compiles to single 'bsr' instruction on x86. - * For zero argument, result is unspecified. - */ -inline unsigned int bitScanReverse(unsigned int x) -{ - assert(x != 0); - return sizeof(unsigned int) * 8 - 1 - __builtin_clz(x); -} - - /** For zero argument, result is zero. * For arguments with most significand bit set, result is n. * For other arguments, returns value, rounded up to power of two. @@ -41,10 +31,9 @@ inline size_t roundUpToPowerOfTwoOrZero(size_t n) template -inline size_t getLeadingZeroBits(T x) +inline size_t getLeadingZeroBitsUnsafe(T x) { - if (!x) - return sizeof(x) * 8; + assert(x != 0); if constexpr (sizeof(T) <= sizeof(unsigned int)) { @@ -60,10 +49,32 @@ inline size_t getLeadingZeroBits(T x) } } + +template +inline size_t getLeadingZeroBits(T x) +{ + if (!x) + return sizeof(x) * 8; + + return getLeadingZeroBitsUnsafe(x); +} + +/** Returns log2 of number, rounded down. + * Compiles to single 'bsr' instruction on x86. + * For zero argument, result is unspecified. + */ +template +inline uint32_t bitScanReverse(T x) +{ + return sizeof(T) * 8 - 1 - getLeadingZeroBitsUnsafe(x); +} + // Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0 template inline size_t getTrailingZeroBitsUnsafe(T x) { + assert(x != 0); + if constexpr (sizeof(T) <= sizeof(unsigned int)) { return __builtin_ctz(x); @@ -88,8 +99,8 @@ inline size_t getTrailingZeroBits(T x) } /** Returns a mask that has '1' for `bits` LSB set: - * maskLowBits(3) => 00000111 - */ + * maskLowBits(3) => 00000111 + */ template inline T maskLowBits(unsigned char bits) { diff --git a/src/Common/StringUtils/StringUtils.h b/src/Common/StringUtils/StringUtils.h index a1e8fb79435..904e3035dd8 100644 --- a/src/Common/StringUtils/StringUtils.h +++ b/src/Common/StringUtils/StringUtils.h @@ -67,10 +67,19 @@ inline bool isASCII(char c) return static_cast(c) < 0x80; } +inline bool isLowerAlphaASCII(char c) +{ + return (c >= 'a' && c <= 'z'); +} + +inline bool isUpperAlphaASCII(char c) +{ + return (c >= 'A' && c <= 'Z'); +} + inline bool isAlphaASCII(char c) { - return (c >= 'a' && c <= 'z') - || (c >= 'A' && c <= 'Z'); + return isLowerAlphaASCII(c) || isUpperAlphaASCII(c); } inline bool isNumericASCII(char c) @@ -122,6 +131,16 @@ inline bool isPrintableASCII(char c) return uc >= 32 && uc <= 126; /// 127 is ASCII DEL. } +inline bool isPunctuationASCII(char c) +{ + uint8_t uc = c; + return (uc >= 33 && uc <= 47) + || (uc >= 58 && uc <= 64) + || (uc >= 91 && uc <= 96) + || (uc >= 123 && uc <= 125); +} + + inline bool isValidIdentifier(const std::string_view & str) { return !str.empty() && isValidIdentifierBegin(str[0]) && std::all_of(str.begin() + 1, str.end(), isWordCharASCII); diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp new file mode 100644 index 00000000000..66221005d77 --- /dev/null +++ b/src/Parsers/obfuscateQueries.cpp @@ -0,0 +1,934 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int TOO_MANY_TEMPORARY_COLUMNS; +} + + +namespace +{ + +const std::unordered_set keywords +{ + "CREATE", "DATABASE", "IF", "NOT", "EXISTS", "TEMPORARY", "TABLE", "ON", "CLUSTER", "DEFAULT", + "MATERIALIZED", "ALIAS", "ENGINE", "AS", "VIEW", "POPULATE", "SETTINGS", "ATTACH", "DETACH", "DROP", + "RENAME", "TO", "ALTER", "ADD", "MODIFY", "CLEAR", "COLUMN", "AFTER", "COPY", "PROJECT", + "PRIMARY", "KEY", "CHECK", "PARTITION", "PART", "FREEZE", "FETCH", "FROM", "SHOW", "INTO", + "OUTFILE", "FORMAT", "TABLES", "DATABASES", "LIKE", "PROCESSLIST", "CASE", "WHEN", "THEN", "ELSE", + "END", "DESCRIBE", "DESC", "USE", "SET", "OPTIMIZE", "FINAL", "DEDUPLICATE", "INSERT", "VALUES", + "SELECT", "DISTINCT", "SAMPLE", "ARRAY", "JOIN", "GLOBAL", "LOCAL", "ANY", "ALL", "INNER", + "LEFT", "RIGHT", "FULL", "OUTER", "CROSS", "USING", "PREWHERE", "WHERE", "GROUP", "BY", + "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", + "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", + "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY" +}; + +const std::unordered_set keep_words +{ + "id", "name", "value", "num", + "Id", "Name", "Value", "Num", + "ID", "NAME", "VALUE", "NUM", +}; + +/// The list of nouns collected from here: http://www.desiquintans.com/nounlist, Public domain. +std::initializer_list nouns +{ +"aardvark", "abacus", "abbey", "abbreviation", "abdomen", "ability", "abnormality", "abolishment", "abortion", +"abrogation", "absence", "abundance", "abuse", "academics", "academy", "accelerant", "accelerator", "accent", "acceptance", "access", +"accessory", "accident", "accommodation", "accompanist", "accomplishment", "accord", "accordance", "accordion", "account", "accountability", +"accountant", "accounting", "accuracy", "accusation", "acetate", "achievement", "achiever", "acid", "acknowledgment", "acorn", "acoustics", +"acquaintance", "acquisition", "acre", "acrylic", "act", "action", "activation", "activist", "activity", "actor", "actress", "acupuncture", +"ad", "adaptation", "adapter", "addiction", "addition", "address", "adjective", "adjustment", "admin", "administration", "administrator", +"admire", "admission", "adobe", "adoption", "adrenalin", "adrenaline", "adult", "adulthood", "advance", "advancement", "advantage", "advent", +"adverb", "advertisement", "advertising", "advice", "adviser", "advocacy", "advocate", "affair", "affect", "affidavit", "affiliate", +"affinity", "afoul", "afterlife", "aftermath", "afternoon", "aftershave", "aftershock", "afterthought", "age", "agency", "agenda", "agent", +"aggradation", "aggression", "aglet", "agony", "agreement", "agriculture", "aid", "aide", "aim", "air", "airbag", "airbus", "aircraft", +"airfare", "airfield", "airforce", "airline", "airmail", "airman", "airplane", "airport", "airship", "airspace", "alarm", "alb", "albatross", +"album", "alcohol", "alcove", "alder", "ale", "alert", "alfalfa", "algebra", "algorithm", "alias", "alibi", "alien", "allegation", "allergist", +"alley", "alliance", "alligator", "allocation", "allowance", "alloy", "alluvium", "almanac", "almighty", "almond", "alpaca", "alpenglow", +"alpenhorn", "alpha", "alphabet", "altar", "alteration", "alternative", "altitude", "alto", "aluminium", "aluminum", "amazement", "amazon", +"ambassador", "amber", "ambience", "ambiguity", "ambition", "ambulance", "amendment", "amenity", "ammunition", "amnesty", "amount", "amusement", +"anagram", "analgesia", "analog", "analogue", "analogy", "analysis", "analyst", "analytics", "anarchist", "anarchy", "anatomy", "ancestor", +"anchovy", "android", "anesthesiologist", "anesthesiology", "angel", "anger", "angina", "angiosperm", "angle", "angora", "angstrom", +"anguish", "animal", "anime", "anise", "ankle", "anklet", "anniversary", "announcement", "annual", "anorak", "answer", "ant", "anteater", +"antecedent", "antechamber", "antelope", "antennae", "anterior", "anthropology", "antibody", "anticipation", "anticodon", "antigen", +"antique", "antiquity", "antler", "antling", "anxiety", "anybody", "anyone", "anything", "anywhere", "apartment", "ape", "aperitif", +"apology", "app", "apparatus", "apparel", "appeal", "appearance", "appellation", "appendix", "appetiser", "appetite", "appetizer", "applause", +"apple", "applewood", "appliance", "application", "appointment", "appreciation", "apprehension", "approach", "appropriation", "approval", +"apricot", "apron", "apse", "aquarium", "aquifer", "arcade", "arch", "archaeologist", "archaeology", "archeology", "archer", +"architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament", +"armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow", +"art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray", +"aside", "asparagus", "aspect", "asphalt", "aspic", "ass", "assassination", "assault", "assembly", "assertion", "assessment", "asset", +"assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe", +"astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment", +"attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney", +"attraction", "attribute", "auction", "audience", "audit", "auditorium", "aunt", "authentication", "authenticity", "author", "authorisation", +"authority", "authorization", "auto", "autoimmunity", "automation", "automaton", "autumn", "availability", "avalanche", "avenue", "average", +"avocado", "award", "awareness", "awe", "axis", "azimuth", "babe", "baboon", "babushka", "baby", "bachelor", "back", "backbone", +"backburn", "backdrop", "background", "backpack", "backup", "backyard", "bacon", "bacterium", "badge", "badger", "bafflement", "bag", +"bagel", "baggage", "baggie", "baggy", "bagpipe", "bail", "bait", "bake", "baker", "bakery", "bakeware", "balaclava", "balalaika", "balance", +"balcony", "ball", "ballet", "balloon", "balloonist", "ballot", "ballpark", "bamboo", "ban", "banana", "band", "bandana", "bandanna", +"bandolier", "bandwidth", "bangle", "banjo", "bank", "bankbook", "banker", "banking", "bankruptcy", "banner", "banquette", "banyan", +"baobab", "bar", "barbecue", "barbeque", "barber", "barbiturate", "bargain", "barge", "baritone", "barium", "bark", "barley", "barn", +"barometer", "barracks", "barrage", "barrel", "barrier", "barstool", "bartender", "base", "baseball", "baseboard", "baseline", "basement", +"basics", "basil", "basin", "basis", "basket", "basketball", "bass", "bassinet", "bassoon", "bat", "bath", "bather", "bathhouse", "bathrobe", +"bathroom", "bathtub", "battalion", "batter", "battery", "batting", "battle", "battleship", "bay", "bayou", "beach", "bead", "beak", +"beam", "bean", "beancurd", "beanie", "beanstalk", "bear", "beard", "beast", "beastie", "beat", "beating", "beauty", "beaver", "beck", +"bed", "bedrock", "bedroom", "bee", "beech", "beef", "beer", "beet", "beetle", "beggar", "beginner", "beginning", "begonia", "behalf", +"behavior", "behaviour", "beheading", "behest", "behold", "being", "belfry", "belief", "believer", "bell", "belligerency", "bellows", +"belly", "belt", "bench", "bend", "beneficiary", "benefit", "beret", "berry", "bestseller", "bet", "beverage", "beyond", +"bias", "bibliography", "bicycle", "bid", "bidder", "bidding", "bidet", "bifocals", "bijou", "bike", "bikini", "bill", "billboard", "billing", +"billion", "bin", "binoculars", "biology", "biopsy", "biosphere", "biplane", "birch", "bird", "birdbath", "birdcage", +"birdhouse", "birth", "birthday", "biscuit", "bit", "bite", "bitten", "bitter", "black", "blackberry", "blackbird", "blackboard", "blackfish", +"blackness", "bladder", "blade", "blame", "blank", "blanket", "blast", "blazer", "blend", "blessing", "blight", "blind", "blinker", "blister", +"blizzard", "block", "blocker", "blog", "blogger", "blood", "bloodflow", "bloom", "bloomer", "blossom", "blouse", "blow", "blowgun", +"blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero", +"bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman", +"book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie", +"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "bottom-line", +"boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler", +"bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet", +"bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit", +"break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick", +"bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli", +"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brother-in-law", +"brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle", +"buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk", +"bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau", +"burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman", +"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "c-clamp", +"cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage", +"cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm", +"calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer", +"candelabra", "candidacy", "candidate", "candle", "candy", "cane", "cannibal", "cannon", "canoe", "canon", "canopy", "cantaloupe", "canteen", +"canvas", "cap", "capability", "capacity", "cape", "caper", "capital", "capitalism", "capitulation", "capon", "cappelletti", "cappuccino", +"captain", "caption", "captor", "car", "carabao", "caramel", "caravan", "carbohydrate", "carbon", "carboxyl", "card", "cardboard", "cardigan", +"care", "career", "cargo", "caribou", "carload", "carnation", "carnival", "carol", "carotene", "carp", "carpenter", "carpet", "carpeting", +"carport", "carriage", "carrier", "carrot", "carry", "cart", "cartel", "carter", "cartilage", "cartload", "cartoon", "cartridge", "carving", +"cascade", "case", "casement", "cash", "cashew", "cashier", "casino", "casket", "cassava", "casserole", "cassock", "cast", "castanet", +"castle", "casualty", "cat", "catacomb", "catalogue", "catalysis", "catalyst", "catamaran", "catastrophe", "catch", "catcher", "category", +"caterpillar", "cathedral", "cation", "catsup", "cattle", "cauliflower", "causal", "cause", "causeway", "caution", "cave", "caviar", +"cayenne", "ceiling", "celebration", "celebrity", "celeriac", "celery", "cell", "cellar", "cello", "celsius", "cement", "cemetery", "cenotaph", +"census", "cent", "center", "centimeter", "centre", "centurion", "century", "cephalopod", "ceramic", "ceramics", "cereal", "ceremony", +"certainty", "certificate", "certification", "cesspool", "chafe", "chain", "chainstay", "chair", "chairlift", "chairman", "chairperson", +"chaise", "chalet", "chalice", "chalk", "challenge", "chamber", "champagne", "champion", "championship", "chance", "chandelier", "change", +"channel", "chaos", "chap", "chapel", "chaplain", "chapter", "character", "characteristic", "characterization", "chard", "charge", "charger", +"charity", "charlatan", "charm", "charset", "chart", "charter", "chasm", "chassis", "chastity", "chasuble", "chateau", "chatter", "chauffeur", +"chauvinist", "check", "checkbook", "checking", "checkout", "checkroom", "cheddar", "cheek", "cheer", "cheese", "cheesecake", "cheetah", +"chef", "chem", "chemical", "chemistry", "chemotaxis", "cheque", "cherry", "chess", "chest", "chestnut", "chick", "chicken", "chicory", +"chief", "chiffonier", "child", "childbirth", "childhood", "chili", "chill", "chime", "chimpanzee", "chin", "chinchilla", "chino", "chip", +"chipmunk", "chivalry", "chive", "chives", "chocolate", "choice", "choir", "choker", "cholesterol", "choosing", "chop", +"chops", "chopstick", "chopsticks", "chord", "chorus", "chow", "chowder", "chrome", "chromolithograph", "chronicle", "chronograph", "chronometer", +"chrysalis", "chub", "chuck", "chug", "church", "churn", "chutney", "cicada", "cigarette", "cilantro", "cinder", "cinema", "cinnamon", +"circadian", "circle", "circuit", "circulation", "circumference", "circumstance", "cirrhosis", "cirrus", "citizen", "citizenship", "citron", +"citrus", "city", "civilian", "civilisation", "civilization", "claim", "clam", "clamp", "clan", "clank", "clapboard", "clarification", +"clarinet", "clarity", "clasp", "class", "classic", "classification", "classmate", "classroom", "clause", "clave", "clavicle", "clavier", +"claw", "clay", "cleaner", "clearance", "clearing", "cleat", "cleavage", "clef", "cleft", "clergyman", "cleric", "clerk", "click", "client", +"cliff", "climate", "climb", "clinic", "clip", "clipboard", "clipper", "cloak", "cloakroom", "clock", "clockwork", "clogs", "cloister", +"clone", "close", "closet", "closing", "closure", "cloth", "clothes", "clothing", "cloud", "cloudburst", "clove", "clover", "cloves", +"club", "clue", "cluster", "clutch", "coach", "coal", "coalition", "coast", "coaster", "coat", "cob", "cobbler", "cobweb", +"cock", "cockpit", "cockroach", "cocktail", "cocoa", "coconut", "cod", "code", "codepage", "codling", "codon", "codpiece", "coevolution", +"cofactor", "coffee", "coffin", "cohesion", "cohort", "coil", "coin", "coincidence", "coinsurance", "coke", "cold", "coleslaw", "coliseum", +"collaboration", "collagen", "collapse", "collar", "collard", "collateral", "colleague", "collection", "collectivisation", "collectivization", +"collector", "college", "collision", "colloquy", "colon", "colonial", "colonialism", "colonisation", "colonization", "colony", "color", +"colorlessness", "colt", "column", "columnist", "comb", "combat", "combination", "combine", "comeback", "comedy", "comestible", "comfort", +"comfortable", "comic", "comics", "comma", "command", "commander", "commandment", "comment", "commerce", "commercial", "commission", +"commitment", "committee", "commodity", "common", "commonsense", "commotion", "communicant", "communication", "communion", "communist", +"community", "commuter", "company", "comparison", "compass", "compassion", "compassionate", "compensation", "competence", "competition", +"competitor", "complaint", "complement", "completion", "complex", "complexity", "compliance", "complication", "complicity", "compliment", +"component", "comportment", "composer", "composite", "composition", "compost", "comprehension", "compress", "compromise", "comptroller", +"compulsion", "computer", "comradeship", "con", "concentrate", "concentration", "concept", "conception", "concern", "concert", "conclusion", +"concrete", "condition", "conditioner", "condominium", "condor", "conduct", "conductor", "cone", "confectionery", "conference", "confidence", +"confidentiality", "configuration", "confirmation", "conflict", "conformation", "confusion", "conga", "congo", "congregation", "congress", +"congressman", "congressperson", "conifer", "connection", "connotation", "conscience", "consciousness", "consensus", "consent", "consequence", +"conservation", "conservative", "consideration", "consignment", "consist", "consistency", "console", "consonant", "conspiracy", "conspirator", +"constant", "constellation", "constitution", "constraint", "construction", "consul", "consulate", "consulting", "consumer", "consumption", +"contact", "contact lens", "contagion", "container", "content", "contention", "contest", "context", "continent", "contingency", "continuity", +"contour", "contract", "contractor", "contrail", "contrary", "contrast", "contribution", "contributor", "control", "controller", "controversy", +"convection", "convenience", "convention", "conversation", "conversion", "convert", "convertible", "conviction", "cook", "cookbook", +"cookie", "cooking", "coonskin", "cooperation", "coordination", "coordinator", "cop", "cope", "copper", "copy", "copying", +"copyright", "copywriter", "coral", "cord", "corduroy", "core", "cork", "cormorant", "corn", "corner", "cornerstone", "cornet", "cornflakes", +"cornmeal", "corporal", "corporation", "corporatism", "corps", "corral", "correspondence", "correspondent", "corridor", "corruption", +"corsage", "cosset", "cost", "costume", "cot", "cottage", "cotton", "couch", "cougar", "cough", "council", "councilman", "councilor", +"councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart", +"counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom", +"cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers", +"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crap", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", +"crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential", +"credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember", +"crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion", +"critic", "criticism", "crocodile", "crocus", "croissant", "crook", "crop", "cross", "crotch", +"croup", "crow", "crowd", "crown", "crucifixion", "crude", "cruelty", "cruise", "crumb", "crunch", "crusader", "crush", "crust", "cry", +"crystal", "crystallography", "cub", "cube", "cuckoo", "cucumber", "cue", "cuisine", "cultivar", "cultivator", "culture", +"culvert", "cummerbund", "cup", "cupboard", "cupcake", "cupola", "curd", "cure", "curio", "curiosity", "curl", "curler", "currant", "currency", +"current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer", +"cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic", +"cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", +"dame", "damn", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", +"data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", +"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", +"deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", +"deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", +"degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den", +"denim", "denominator", "density", "dentist", "deodorant", "department", "departure", "dependency", "dependent", "deployment", "deposit", +"deposition", "depot", "depression", "depressive", "depth", "deputy", "derby", "derivation", "derivative", "derrick", "descendant", "descent", +"description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer", +"destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation", +"developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem", +"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dick", +"dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", +"dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple", +"diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director", +"directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge", +"discipline", "disclaimer", "disclosure", "disco", "disconnection", "discount", "discourse", "discovery", "discrepancy", "discretion", +"discrimination", "discussion", "disdain", "disease", "disembodiment", "disengagement", "disguise", "disgust", "dish", "dishwasher", +"disk", "disparity", "dispatch", "displacement", "display", "disposal", "disposer", "disposition", "dispute", "disregard", "disruption", +"dissemination", "dissonance", "distance", "distinction", "distortion", "distribution", "distributor", "district", "divalent", "divan", +"diver", "diversity", "divide", "dividend", "divider", "divine", "diving", "division", "divorce", "doc", "dock", "doctor", "doctorate", +"doctrine", "document", "documentary", "documentation", "doe", "dog", "doggie", "dogsled", "dogwood", "doing", "doll", "dollar", "dollop", +"dolman", "dolor", "dolphin", "domain", "dome", "domination", "donation", "donkey", "donor", "donut", "door", "doorbell", "doorknob", +"doorpost", "doorway", "dory", "dose", "dot", "double", "doubling", "doubt", "doubter", "dough", "doughnut", "down", "downfall", "downforce", +"downgrade", "download", "downstairs", "downtown", "downturn", "dozen", "draft", "drag", "dragon", "dragonfly", "dragonfruit", "dragster", +"drain", "drainage", "drake", "drama", "dramaturge", "drapes", "draw", "drawbridge", "drawer", "drawing", "dream", "dreamer", "dredger", +"dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop", +"drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout", +"dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk", +"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "e-book", +"eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake", +"earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics", +"economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial", +"education", "eel", "effacement", "effect", "effective", "effectiveness", "effector", "efficacy", "efficiency", "effort", "egg", "egghead", +"eggnog", "eggplant", "ego", "eicosanoid", "ejector", "elbow", "elderberry", "election", "electricity", "electrocardiogram", "electronics", +"element", "elephant", "elevation", "elevator", "eleventh", "elf", "elicit", "eligibility", "elimination", "elite", "elixir", "elk", +"ellipse", "elm", "elongation", "elver", "email", "emanate", "embarrassment", "embassy", "embellishment", "embossing", "embryo", "emerald", +"emergence", "emergency", "emergent", "emery", "emission", "emitter", "emotion", "emphasis", "empire", "employ", "employee", "employer", +"employment", "empowerment", "emu", "enactment", "encirclement", "enclave", "enclosure", "encounter", "encouragement", "encyclopedia", +"end", "endive", "endoderm", "endorsement", "endothelium", "endpoint", "enemy", "energy", "enforcement", "engagement", "engine", "engineer", +"engineering", "enigma", "enjoyment", "enquiry", "enrollment", "enterprise", "entertainment", "enthusiasm", "entirety", "entity", "entrance", +"entree", "entrepreneur", "entry", "envelope", "environment", "envy", "enzyme", "epauliere", "epee", "ephemera", "ephemeris", "ephyra", +"epic", "episode", "epithelium", "epoch", "eponym", "epoxy", "equal", "equality", "equation", "equinox", "equipment", "equity", "equivalent", +"era", "eraser", "erection", "erosion", "error", "escalator", "escape", "escort", "espadrille", "espalier", "essay", "essence", "essential", +"establishment", "estate", "estimate", "estrogen", "estuary", "eternity", "ethernet", "ethics", "ethnicity", "ethyl", "euphonium", "eurocentrism", +"evaluation", "evaluator", "evaporation", "eve", "evening", "event", "everybody", "everyone", "everything", "eviction", +"evidence", "evil", "evocation", "evolution", "exaggeration", "exam", "examination", "examiner", "example", +"exasperation", "excellence", "exception", "excerpt", "excess", "exchange", "excitement", "exclamation", "excursion", "excuse", "execution", +"executive", "executor", "exercise", "exhaust", "exhaustion", "exhibit", "exhibition", "exile", "existence", "exit", "exocrine", "expansion", +"expansionism", "expectancy", "expectation", "expedition", "expense", "experience", "experiment", "experimentation", "expert", "expertise", +"explanation", "exploration", "explorer", "explosion", "export", "expose", "exposition", "exposure", "expression", "extension", "extent", +"exterior", "external", "extinction", "extreme", "extremist", "eye", "eyeball", "eyebrow", "eyebrows", "eyeglasses", "eyelash", "eyelashes", +"eyelid", "eyelids", "eyeliner", "eyestrain", "eyrie", "fabric", "face", "facelift", "facet", "facility", "facsimile", "fact", "factor", +"factory", "faculty", "fahrenheit", "fail", "failure", "fairness", "fairy", "faith", "faithful", "fall", "fallacy", "fame", +"familiar", "familiarity", "family", "fan", "fang", "fanlight", "fanny", "fantasy", "farm", "farmer", "farming", "farmland", +"farrow", "fascia", "fashion", "fat", "fate", "father", "fatigue", "fatigues", "faucet", "fault", "fav", "fava", "favor", +"favorite", "fawn", "fax", "fear", "feast", "feather", "feature", "fedelini", "federation", "fedora", "fee", "feed", "feedback", "feeding", +"feel", "feeling", "fellow", "felony", "female", "fen", "fence", "fencing", "fender", "feng", "fennel", "ferret", "ferry", "ferryboat", +"fertilizer", "festival", "fetus", "few", "fiber", "fiberglass", "fibre", "fibroblast", "fibrosis", "ficlet", "fiction", "fiddle", "field", +"fiery", "fiesta", "fifth", "fig", "fight", "fighter", "figure", "figurine", "file", "filing", "fill", "fillet", "filly", "film", "filter", +"filth", "final", "finance", "financing", "finding", "fine", "finer", "finger", "fingerling", "fingernail", "finish", "finisher", "fir", +"fire", "fireman", "fireplace", "firewall", "firm", "first", "fish", "fishbone", "fisherman", "fishery", "fishing", "fishmonger", "fishnet", +"fisting", "fit", "fitness", "fix", "fixture", "flag", "flair", "flame", "flan", "flanker", "flare", "flash", "flat", "flatboat", "flavor", +"flax", "fleck", "fledgling", "fleece", "flesh", "flexibility", "flick", "flicker", "flight", "flint", "flintlock", "flock", +"flood", "floodplain", "floor", "floozie", "flour", "flow", "flower", "flu", "flugelhorn", "fluke", "flume", "flung", "flute", "fly", +"flytrap", "foal", "foam", "fob", "focus", "fog", "fold", "folder", "folk", "folklore", "follower", "following", "fondue", "font", "food", +"foodstuffs", "fool", "foot", "footage", "football", "footnote", "footprint", "footrest", "footstep", "footstool", "footwear", "forage", +"forager", "foray", "force", "ford", "forearm", "forebear", "forecast", "forehead", "foreigner", "forelimb", "forest", "forestry", "forever", +"forgery", "fork", "form", "formal", "formamide", "format", "formation", "former", "formicarium", "formula", "fort", "forte", "fortnight", +"fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance", +"frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight", +"freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe", +"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuck", "fuel", "fugato", +"fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture", +"furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley", +"gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden", +"garlic", "garment", "garter", "gas", "gasket", "gasoline", "gasp", "gastronomy", "gastropod", "gate", "gateway", "gather", "gathering", +"gator", "gauge", "gauntlet", "gavel", "gazebo", "gazelle", "gear", "gearshift", "geek", "gel", "gelatin", "gelding", "gem", "gemsbok", +"gender", "gene", "general", "generation", "generator", "generosity", "genetics", "genie", "genius", "genocide", "genre", "gentleman", +"geography", "geology", "geometry", "geranium", "gerbil", "gesture", "geyser", "gherkin", "ghost", "giant", "gift", "gig", "gigantism", +"giggle", "ginger", "gingerbread", "ginseng", "giraffe", "girdle", "girl", "girlfriend", "git", "glacier", "gladiolus", "glance", "gland", +"glass", "glasses", "glee", "glen", "glider", "gliding", "glimpse", "globe", "glockenspiel", "gloom", "glory", "glove", "glow", "glucose", +"glue", "glut", "glutamate", "gnat", "gnu", "goal", "goat", "gobbler", "god", "goddess", "godfather", "godmother", "godparent", +"goggles", "going", "gold", "goldfish", "golf", "gondola", "gong", "good", "goodbye", "goodie", "goodness", "goodnight", +"goodwill", "goose", "gopher", "gorilla", "gosling", "gossip", "governance", "government", "governor", "gown", "grace", "grade", +"gradient", "graduate", "graduation", "graffiti", "graft", "grain", "gram", "grammar", "gran", "grand", "grandchild", "granddaughter", +"grandfather", "grandma", "grandmom", "grandmother", "grandpa", "grandparent", "grandson", "granny", "granola", "grant", "grape", "grapefruit", +"graph", "graphic", "grasp", "grass", "grasshopper", "grassland", "gratitude", "gravel", "gravitas", "gravity", "gravy", "gray", "grease", +"greatness", "greed", "green", "greenhouse", "greens", "grenade", "grey", "grid", "grief", +"grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole", +"guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty", +"guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology", +"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "half-sister", +"halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball", +"handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness", +"harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm", +"harmonica", "harmonise", "harmonize", "harmony", "harp", "harpooner", "harpsichord", "harvest", "harvester", "hash", "hashtag", "hassock", +"haste", "hat", "hatbox", "hatchet", "hatchling", "hate", "hatred", "haunt", "haven", "haversack", "havoc", "hawk", "hay", "haze", "hazel", +"hazelnut", "head", "headache", "headlight", "headline", "headphones", "headquarters", "headrest", "health", "hearing", +"hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven", +"heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello", +"helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron", +"herring", "hesitation", "heterosexual", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", +"highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive", +"hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework", +"homicide", "homogenate", "homonym", "homosexual", "homosexuality", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", +"hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose", +"hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel", +"hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard", +"howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback", +"hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth", +"hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis", +"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "ice-cream", +"iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology", +"idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image", +"imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement", +"implementation", "implication", "import", "importance", "impostor", "impress", "impression", "imprisonment", "impropriety", "improvement", +"impudence", "impulse", "inability", "inauguration", "inbox", "incandescence", "incarnation", "incense", "incentive", +"inch", "incidence", "incident", "incision", "inclusion", "income", "incompetence", "inconvenience", "increase", "incubation", "independence", +"independent", "index", "indication", "indicator", "indigence", "individual", "industrialisation", "industrialization", "industry", "inequality", +"inevitable", "infancy", "infant", "infarction", "infection", "infiltration", "infinite", "infix", "inflammation", "inflation", "influence", +"influx", "info", "information", "infrastructure", "infusion", "inglenook", "ingrate", "ingredient", "inhabitant", "inheritance", "inhibition", +"inhibitor", "initial", "initialise", "initialize", "initiative", "injunction", "injury", "injustice", "ink", "inlay", "inn", "innervation", +"innocence", "innocent", "innovation", "input", "inquiry", "inscription", "insect", "insectarium", "insert", "inside", "insight", "insolence", +"insomnia", "inspection", "inspector", "inspiration", "installation", "instance", "instant", "instinct", "institute", "institution", +"instruction", "instructor", "instrument", "instrumentalist", "instrumentation", "insulation", "insurance", "insurgence", "insurrection", +"integer", "integral", "integration", "integrity", "intellect", "intelligence", "intensity", "intent", "intention", "intentionality", +"interaction", "interchange", "interconnection", "intercourse", "interest", "interface", "interferometer", "interior", "interject", "interloper", +"internet", "interpretation", "interpreter", "interval", "intervenor", "intervention", "interview", "interviewer", "intestine", "introduction", +"intuition", "invader", "invasion", "invention", "inventor", "inventory", "inverse", "inversion", "investigation", "investigator", "investment", +"investor", "invitation", "invite", "invoice", "involvement", "iridescence", "iris", "iron", "ironclad", "irony", "irrigation", "ischemia", +"island", "isogloss", "isolation", "issue", "item", "itinerary", "ivory", "jack", "jackal", "jacket", "jackfruit", "jade", "jaguar", +"jail", "jailhouse", "jalapeño", "jam", "jar", "jasmine", "jaw", "jazz", "jealousy", "jeans", "jeep", "jelly", "jellybeans", "jellyfish", +"jerk", "jet", "jewel", "jeweller", "jewellery", "jewelry", "jicama", "jiffy", "job", "jockey", "jodhpurs", "joey", "jogging", "joint", +"joke", "jot", "journal", "journalism", "journalist", "journey", "joy", "judge", "judgment", "judo", "jug", "juggernaut", "juice", "julienne", +"jumbo", "jump", "jumper", "jumpsuit", "jungle", "junior", "junk", "junker", "junket", "jury", "justice", "justification", "jute", "kale", +"kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum", +"key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram", +"kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite", +"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "know-how", +"knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace", +"lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land", +"landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard", +"larder", "lark", "larva", "laryngitis", "lasagna", "lashes", "last", "latency", "latex", "lathe", "latitude", "latte", "latter", "laugh", +"laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership", +"leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left", +"leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon", +"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesbian", "lesson", "letter", +"lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing", +"licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning", +"lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line", +"linen", "liner", "linguist", "linguistics", "lining", "link", "linkage", "linseed", "lion", "lip", "lipid", "lipoprotein", "lipstick", +"liquid", "liquidity", "liquor", "list", "listening", "listing", "literate", "literature", "litigation", "litmus", "litter", "littleneck", +"liver", "livestock", "living", "lizard", "llama", "load", "loading", "loaf", "loafer", "loan", "lobby", "lobotomy", "lobster", "local", +"locality", "location", "lock", "locker", "locket", "locomotive", "locust", "lode", "loft", "log", "loggia", "logic", "login", "logistics", +"logo", "loincloth", "lollipop", "loneliness", "longboat", "longitude", "look", "lookout", "loop", "loophole", "loquat", "lord", "loss", +"lot", "lotion", "lottery", "lounge", "louse", "lout", "love", "lover", "lox", "loyalty", "luck", "luggage", "lumber", "lumberman", "lunch", +"luncheonette", "lunchmeat", "lunchroom", "lung", "lunge", "lust", "lute", "luxury", "lychee", "lycra", "lye", "lymphocyte", "lynx", +"lyocell", "lyre", "lyrics", "lysine", "mRNA", "macadamia", "macaroni", "macaroon", "macaw", "machine", "machinery", "macrame", "macro", +"macrofauna", "madam", "maelstrom", "maestro", "magazine", "maggot", "magic", "magnet", "magnitude", "maid", "maiden", "mail", "mailbox", +"mailer", "mailing", "mailman", "main", "mainland", "mainstream", "maintainer", "maintenance", "maize", "major", "majority", +"makeover", "maker", "makeup", "making", "male", "malice", "mall", "mallard", "mallet", "malnutrition", "mama", "mambo", "mammoth", "man", +"manacle", "management", "manager", "manatee", "mandarin", "mandate", "mandolin", "mangle", "mango", "mangrove", "manhunt", "maniac", +"manicure", "manifestation", "manipulation", "mankind", "manner", "manor", "mansard", "manservant", "mansion", "mantel", "mantle", "mantua", +"manufacturer", "manufacturing", "many", "map", "maple", "mapping", "maracas", "marathon", "marble", "march", "mare", "margarine", "margin", +"mariachi", "marimba", "marines", "marionberry", "mark", "marker", "market", "marketer", "marketing", "marketplace", "marksman", "markup", +"marmalade", "marriage", "marsh", "marshland", "marshmallow", "marten", "marxism", "mascara", "mask", "masonry", "mass", "massage", "mast", +"master", "masterpiece", "mastication", "mastoid", "mat", "match", "matchmaker", "mate", "material", "maternity", "math", "mathematics", +"matrix", "matter", "mattock", "mattress", "max", "maximum", "maybe", "mayonnaise", "mayor", "meadow", "meal", "mean", "meander", "meaning", +"means", "meantime", "measles", "measure", "measurement", "meat", "meatball", "meatloaf", "mecca", "mechanic", "mechanism", "med", "medal", +"media", "median", "medication", "medicine", "medium", "meet", "meeting", "melatonin", "melody", "melon", "member", "membership", "membrane", +"meme", "memo", "memorial", "memory", "men", "menopause", "menorah", "mention", "mentor", "menu", "merchandise", "merchant", "mercury", +"meridian", "meringue", "merit", "mesenchyme", "mess", "message", "messenger", "messy", "metabolite", "metal", "metallurgist", "metaphor", +"meteor", "meteorology", "meter", "methane", "method", "methodology", "metric", "metro", "metronome", "mezzanine", "microlending", "micronutrient", +"microphone", "microwave", "midden", "middle", "middleman", "midline", "midnight", "midwife", "might", "migrant", "migration", +"mile", "mileage", "milepost", "milestone", "military", "milk", "milkshake", "mill", "millennium", "millet", "millimeter", "million", +"millisecond", "millstone", "mime", "mimosa", "min", "mincemeat", "mind", "mine", "mineral", "mineshaft", "mini", "minibus", +"minimalism", "minimum", "mining", "minion", "minister", "mink", "minnow", "minor", "minority", "mint", "minute", "miracle", +"mirror", "miscarriage", "miscommunication", "misfit", "misnomer", "misogyny", "misplacement", "misreading", "misrepresentation", "miss", +"missile", "mission", "missionary", "mist", "mistake", "mister", "misunderstand", "miter", "mitten", "mix", "mixer", "mixture", "moai", +"moat", "mob", "mobile", "mobility", "mobster", "moccasins", "mocha", "mochi", "mode", "model", "modeling", "modem", "modernist", "modernity", +"modification", "molar", "molasses", "molding", "mole", "molecule", "mom", "moment", "monastery", "monasticism", "money", "monger", "monitor", +"monitoring", "monk", "monkey", "monocle", "monopoly", "monotheism", "monsoon", "monster", "month", "monument", "mood", "moody", "moon", +"moonlight", "moonscape", "moonshine", "moose", "mop", "morale", "morbid", "morbidity", "morning", "moron", "morphology", "morsel", "mortal", +"mortality", "mortgage", "mortise", "mosque", "mosquito", "most", "motel", "moth", "mother", "motion", "motivation", +"motive", "motor", "motorboat", "motorcar", "motorcycle", "mound", "mountain", "mouse", "mouser", "mousse", "moustache", "mouth", "mouton", +"movement", "mover", "movie", "mower", "mozzarella", "mud", "muffin", "mug", "mukluk", "mule", "multimedia", "murder", "muscat", "muscatel", +"muscle", "musculature", "museum", "mushroom", "music", "musician", "muskrat", "mussel", "mustache", "mustard", +"mutation", "mutt", "mutton", "mycoplasma", "mystery", "myth", "mythology", "nail", "name", "naming", "nanoparticle", "napkin", "narrative", +"nasal", "nation", "nationality", "native", "naturalisation", "nature", "navigation", "necessity", "neck", "necklace", "necktie", "nectar", +"nectarine", "need", "needle", "neglect", "negligee", "negotiation", "neighbor", "neighborhood", "neighbour", "neighbourhood", "neologism", +"neon", "neonate", "nephew", "nerve", "nest", "nestling", "nestmate", "net", "netball", "netbook", "netsuke", "network", "networking", +"neurobiologist", "neuron", "neuropathologist", "neuropsychiatry", "news", "newsletter", "newspaper", "newsprint", "newsstand", "nexus", +"nibble", "nicety", "niche", "nick", "nickel", "nickname", "niece", "night", "nightclub", "nightgown", "nightingale", "nightlife", "nightlight", +"nightmare", "ninja", "nit", "nitrogen", "nobody", "nod", "node", "noir", "noise", "nonbeliever", "nonconformist", "nondisclosure", "nonsense", +"noodle", "noodles", "noon", "norm", "normal", "normalisation", "normalization", "north", "nose", "notation", "note", "notebook", "notepad", +"nothing", "notice", "notion", "notoriety", "nougat", "noun", "nourishment", "novel", "nucleotidase", "nucleotide", "nudge", "nuke", +"number", "numeracy", "numeric", "numismatist", "nun", "nurse", "nursery", "nursing", "nurture", "nut", "nutmeg", "nutrient", "nutrition", +"nylon", "nymph", "oak", "oar", "oasis", "oat", "oatmeal", "oats", "obedience", "obesity", "obi", "object", "objection", "objective", +"obligation", "oboe", "observation", "observatory", "obsession", "obsidian", "obstacle", "occasion", "occupation", "occurrence", "ocean", +"ocelot", "octagon", "octave", "octavo", "octet", "octopus", "odometer", "odyssey", "oeuvre", "offence", "offense", "offer", +"offering", "office", "officer", "official", "offset", "oil", "okra", "oldie", "oleo", "olive", "omega", "omelet", "omission", "omnivore", +"oncology", "onion", "online", "onset", "opening", "opera", "operating", "operation", "operator", "ophthalmologist", "opinion", "opium", +"opossum", "opponent", "opportunist", "opportunity", "opposite", "opposition", "optimal", "optimisation", "optimist", "optimization", +"option", "orange", "orangutan", "orator", "orchard", "orchestra", "orchid", "order", "ordinary", "ordination", "ore", "oregano", "organ", +"organisation", "organising", "organization", "organizing", "orient", "orientation", "origin", "original", "originality", "ornament", +"osmosis", "osprey", "ostrich", "other", "otter", "ottoman", "ounce", "outback", "outcome", "outfielder", "outfit", "outhouse", "outlaw", +"outlay", "outlet", "outline", "outlook", "output", "outrage", "outrigger", "outrun", "outset", "outside", "oval", "ovary", "oven", "overcharge", +"overclocking", "overcoat", "overexertion", "overflight", "overhead", "overheard", "overload", "overnighter", "overshoot", "oversight", +"overview", "overweight", "owl", "owner", "ownership", "ox", "oxford", "oxygen", "oyster", "ozone", "pace", "pacemaker", "pack", "package", +"packaging", "packet", "pad", "paddle", "paddock", "pagan", "page", "pagoda", "pail", "pain", "paint", "painter", "painting", "paintwork", +"pair", "pajamas", "palace", "palate", "palm", "pamphlet", "pan", "pancake", "pancreas", "panda", "panel", "panic", "pannier", "panpipe", +"pansy", "panther", "panties", "pantologist", "pantology", "pantry", "pants", "pantsuit", "panty", "pantyhose", "papa", "papaya", "paper", +"paperback", "paperwork", "parable", "parachute", "parade", "paradise", "paragraph", "parallelogram", "paramecium", "paramedic", "parameter", +"paranoia", "parcel", "parchment", "pard", "pardon", "parent", "parenthesis", "parenting", "park", "parka", "parking", "parliament", +"parole", "parrot", "parser", "parsley", "parsnip", "part", "participant", "participation", "particle", "particular", "partner", "partnership", +"partridge", "party", "pass", "passage", "passbook", "passenger", "passing", "passion", "passive", "passport", "password", "past", "pasta", +"paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology", +"pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron", +"pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace", +"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "peer-to-peer", +"pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis", +"pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch", +"perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp", +"perpendicular", "persimmon", "person", "personal", "personality", "personnel", "perspective", "pest", "pet", "petal", "petition", "petitioner", +"petticoat", "pew", "pharmacist", "pharmacopoeia", "phase", "pheasant", "phenomenon", "phenotype", "pheromone", "philanthropy", "philosopher", +"philosophy", "phone", "phosphate", "photo", "photodiode", "photograph", "photographer", "photography", "photoreceptor", "phrase", "phrasing", +"physical", "physics", "physiology", "pianist", "piano", "piccolo", "pick", "pickax", "pickaxe", "picket", "pickle", "pickup", "picnic", +"picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim", +"pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple", +"pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate", +"piss", "pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", +"plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard", +"plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright", +"plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug", +"plugin", "plum", "plumber", "plume", "plunger", "plywood", "pneumonia", "pocket", "pocketbook", "pod", "podcast", "poem", +"poet", "poetry", "poignance", "point", "poison", "poisoning", "poker", "polarisation", "polarization", "pole", "polenta", "police", +"policeman", "policy", "polish", "politician", "politics", "poll", "polliwog", "pollutant", "pollution", "polo", "polyester", "polyp", +"pomegranate", "pomelo", "pompom", "poncho", "pond", "pony", "pool", "poor", "pop", "popcorn", "poppy", "popsicle", "popularity", "population", +"populist", "porcelain", "porch", "porcupine", "pork", "porpoise", "port", "porter", "portfolio", "porthole", "portion", "portrait", +"position", "possession", "possibility", "possible", "post", "postage", "postbox", "poster", "posterior", "postfix", "pot", "potato", +"potential", "pottery", "potty", "pouch", "poultry", "pound", "pounding", "poverty", "powder", "power", "practice", "practitioner", "prairie", +"praise", "pray", "prayer", "precedence", "precedent", "precipitation", "precision", "predecessor", "preface", "preference", "prefix", +"pregnancy", "prejudice", "prelude", "premeditation", "premier", "premise", "premium", "preoccupation", "preparation", "prescription", +"presence", "present", "presentation", "preservation", "preserves", "presidency", "president", "press", "pressroom", "pressure", "pressurisation", +"pressurization", "prestige", "presume", "pretzel", "prevalence", "prevention", "prey", "price", "pricing", "pride", "priest", "priesthood", +"primary", "primate", "prince", "princess", "principal", "principle", "print", "printer", "printing", "prior", "priority", "prison", +"prisoner", "privacy", "private", "privilege", "prize", "prizefight", "probability", "probation", "probe", "problem", "procedure", "proceedings", +"process", "processing", "processor", "proctor", "procurement", "produce", "producer", "product", "production", "productivity", "profession", +"professional", "professor", "profile", "profit", "progenitor", "program", "programme", "programming", "progress", "progression", "prohibition", +"project", "proliferation", "promenade", "promise", "promotion", "prompt", "pronoun", "pronunciation", "proof", "propaganda", +"propane", "property", "prophet", "proponent", "proportion", "proposal", "proposition", "proprietor", "prose", "prosecution", "prosecutor", +"prospect", "prosperity", "prostacyclin", "prostanoid", "prostrate", "protection", "protein", "protest", "protocol", "providence", "provider", +"province", "provision", "prow", "proximal", "proximity", "prune", "pruner", "pseudocode", "pseudoscience", "psychiatrist", "psychoanalyst", +"psychologist", "psychology", "ptarmigan", "pub", "public", "publication", "publicity", "publisher", "publishing", "pudding", "puddle", +"puffin", "pug", "puggle", "pulley", "pulse", "puma", "pump", "pumpernickel", "pumpkin", "pumpkinseed", "pun", "punch", "punctuation", +"punishment", "pup", "pupa", "pupil", "puppet", "puppy", "purchase", "puritan", "purity", "purple", "purpose", "purr", "purse", "pursuit", +"push", "pusher", "put", "puzzle", "pyramid", "pyridine", "quadrant", "quail", "qualification", "quality", "quantity", "quart", "quarter", +"quartet", "quartz", "queen", "query", "quest", "question", "questioner", "questionnaire", "quiche", "quicksand", "quiet", "quill", "quilt", +"quince", "quinoa", "quit", "quiver", "quota", "quotation", "quote", "rabbi", "rabbit", "raccoon", "race", "racer", "racing", "racism", +"racist", "rack", "radar", "radiator", "radio", "radiosonde", "radish", "raffle", "raft", "rag", "rage", "raid", "rail", "railing", "railroad", +"railway", "raiment", "rain", "rainbow", "raincoat", "rainmaker", "rainstorm", "rainy", "raise", "raisin", "rake", "rally", "ram", "rambler", +"ramen", "ramie", "ranch", "rancher", "randomisation", "randomization", "range", "ranger", "rank", "rap", "rape", "raspberry", "rat", +"rate", "ratepayer", "rating", "ratio", "rationale", "rations", "raven", "ravioli", "rawhide", "ray", "rayon", "razor", "reach", "reactant", +"reaction", "read", "reader", "readiness", "reading", "real", "reality", "realization", "realm", "reamer", "rear", "reason", "reasoning", +"rebel", "rebellion", "reboot", "recall", "recapitulation", "receipt", "receiver", "reception", "receptor", "recess", "recession", "recipe", +"recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record", +"recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction", +"reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal", +"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalisation", +"rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release", +"reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance", +"rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply", +"report", "reporter", "reporting", "repository", "representation", "representative", "reprocessing", "republic", "republican", "reputation", +"request", "requirement", "resale", "rescue", "research", "researcher", "resemblance", "reservation", "reserve", "reservoir", "reset", +"residence", "resident", "residue", "resist", "resistance", "resolution", "resolve", "resort", "resource", "respect", "respite", "response", +"responsibility", "rest", "restaurant", "restoration", "restriction", "restroom", "restructuring", "result", "resume", "retailer", "retention", +"rethinking", "retina", "retirement", "retouching", "retreat", "retrospect", "retrospective", "retrospectivity", "return", "reunion", +"revascularisation", "revascularization", "reveal", "revelation", "revenant", "revenge", "revenue", "reversal", "reverse", "review", +"revitalisation", "revitalization", "revival", "revolution", "revolver", "reward", "rhetoric", "rheumatism", "rhinoceros", "rhubarb", +"rhyme", "rhythm", "rib", "ribbon", "rice", "riddle", "ride", "rider", "ridge", "riding", "rifle", "right", "rim", "ring", "ringworm", +"riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast", +"robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance", +"roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route", +"router", "routine", "row", "rowboat", "rowing", "rubber", "rubbish", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", +"ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye", +"sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing", +"sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar", +"sampan", "sample", "samurai", "sanction", "sanctity", "sanctuary", "sand", "sandal", "sandbar", "sandpaper", "sandwich", "sanity", "sardine", +"sari", "sarong", "sash", "satellite", "satin", "satire", "satisfaction", "sauce", "saucer", "sauerkraut", "sausage", "savage", "savannah", +"saving", "savings", "savior", "saviour", "savory", "saw", "saxophone", "scaffold", "scale", "scallion", "scallops", "scalp", "scam", +"scanner", "scarecrow", "scarf", "scarification", "scenario", "scene", "scenery", "scent", "schedule", "scheduling", "schema", "scheme", +"schizophrenic", "schnitzel", "scholar", "scholarship", "school", "schoolhouse", "schooner", "science", "scientist", "scimitar", "scissors", +"scooter", "scope", "score", "scorn", "scorpion", "scotch", "scout", "scow", "scrambled", "scrap", "scraper", "scratch", "screamer", +"screen", "screening", "screenwriting", "screw", "screwdriver", "scrim", "scrip", "script", "scripture", "scrutiny", "sculpting", +"sculptural", "sculpture", "sea", "seabass", "seafood", "seagull", "seal", "seaplane", "search", "seashore", "seaside", "season", "seat", +"seaweed", "second", "secrecy", "secret", "secretariat", "secretary", "secretion", "section", "sectional", "sector", "security", "sediment", +"seed", "seeder", "seeker", "seep", "segment", "seizure", "selection", "self", "seller", +"selling", "semantics", "semester", "semicircle", "semicolon", "semiconductor", "seminar", "senate", "senator", "sender", "senior", "sense", +"sensibility", "sensitive", "sensitivity", "sensor", "sentence", "sentencing", "sentiment", "sepal", "separation", "septicaemia", "sequel", +"sequence", "serial", "series", "sermon", "serum", "serval", "servant", "server", "service", "servitude", "sesame", "session", "set", +"setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", +"shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", +"shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shit", "shoat", "shock", "shoe", "shoe-horn", +"shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", +"short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", +"shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", +"siege", "sigh", "sight", "sightseeing", "sign", "signal", "signature", "signet", "significance", "signify", "signup", "silence", "silica", +"silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin", +"sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size", +"skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky", +"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slave", "slavery", "slaw", "sled", "sledge", +"sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot", +"sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail", +"snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake", +"snowman", "snowmobiling", "snowplow", "snowstorm", "snowsuit", "snuck", "snug", "snuggle", "soap", "soccer", "socialism", "socialist", +"society", "sociology", "sock", "socks", "soda", "sofa", "softball", "softdrink", "softening", "software", "soil", "soldier", "sole", +"solicitation", "solicitor", "solidarity", "solidity", "soliloquy", "solitaire", "solution", "solvency", "sombrero", "somebody", "someone", +"someplace", "somersault", "something", "somewhere", "son", "sonar", "sonata", "song", "songbird", "sonnet", "soot", "sophomore", "soprano", +"sorbet", "sorghum", "sorrel", "sorrow", "sort", "soul", "soulmate", "sound", "soundness", "soup", "source", "sourwood", "sousaphone", +"south", "southeast", "souvenir", "sovereignty", "sow", "soy", "soybean", "space", "spacing", "spade", "spaghetti", "span", "spandex", +"spank", "sparerib", "spark", "sparrow", "spasm", "spat", "spatula", "spawn", "speaker", "speakerphone", "speaking", "spear", "spec", +"special", "specialist", "specialty", "species", "specification", "spectacle", "spectacles", "spectrograph", "spectrum", "speculation", +"speech", "speed", "speedboat", "spell", "spelling", "spelt", "spending", "sphere", "sphynx", "spice", "spider", "spiderling", "spike", +"spill", "spinach", "spine", "spiral", "spirit", "spiritual", "spirituality", "spit", "spite", "spleen", "splendor", "split", "spokesman", +"spokeswoman", "sponge", "sponsor", "sponsorship", "spool", "spoon", "spork", "sport", "sportsman", "spot", "spotlight", "spouse", "sprag", +"sprat", "spray", "spread", "spreadsheet", "spree", "spring", "sprinkles", "sprinter", "sprout", "spruce", "spud", "spume", "spur", "spy", +"spyglass", "square", "squash", "squatter", "squeegee", "squid", "squirrel", "stab", "stability", "stable", "stack", "stacking", "stadium", +"staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance", +"stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter", +"state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak", +"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "step-daughter", +"stepdaughter", "stepmother", +"stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting", +"stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool", +"stop", "stopsign", "stopwatch", "storage", "store", "storey", "storm", "story", "storyboard", "stot", "stove", "strait", +"strand", "stranger", "strap", "strategy", "straw", "strawberry", "strawman", "stream", "street", "streetcar", "strength", "stress", +"stretch", "strife", "strike", "string", "strip", "stripe", "strobe", "stroke", "structure", "strudel", "struggle", "stucco", "stud", +"student", "studio", "study", "stuff", "stumbling", "stump", "stupidity", "sturgeon", "sty", "style", "styling", "stylus", "sub", "subcomponent", +"subconscious", "subcontractor", "subexpression", "subgroup", "subject", "submarine", "submitter", "subprime", "subroutine", "subscription", +"subsection", "subset", "subsidence", "subsidiary", "subsidy", "substance", "substitution", "subtitle", "suburb", "subway", "success", +"succotash", "suck", "sucker", "suede", "suet", "suffocation", "sugar", "suggestion", "suicide", "suit", "suitcase", "suite", "sulfur", +"sultan", "sum", "summary", "summer", "summit", "sun", "sunbeam", "sunbonnet", "sundae", "sunday", "sundial", "sunflower", "sunglasses", +"sunlamp", "sunlight", "sunrise", "sunroom", "sunset", "sunshine", "superiority", "supermarket", "supernatural", "supervision", "supervisor", +"supper", "supplement", "supplier", "supply", "support", "supporter", "suppression", "supreme", "surface", "surfboard", "surge", "surgeon", +"surgery", "surname", "surplus", "surprise", "surround", "surroundings", "surrounds", "survey", "survival", "survivor", "sushi", "suspect", +"suspenders", "suspension", "sustainment", "sustenance", "swallow", "swamp", "swan", "swanling", "swath", "sweat", "sweater", "sweatshirt", +"sweatshop", "sweatsuit", "sweets", "swell", "swim", "swimming", "swimsuit", "swine", "swing", "switch", "switchboard", "switching", +"swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy", +"synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop", +"tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover", +"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tank-top", +"tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler", +"tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech", +"technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype", +"television", "tell", "teller", "temp", "temper", "temperature", "temple", "tempo", "temporariness", "temporary", "temptation", "temptress", +"tenant", "tendency", "tender", "tenement", "tenet", "tennis", "tenor", "tension", "tensor", "tent", "tentacle", "tenth", "tepee", "teriyaki", +"term", "terminal", "termination", "terminology", "termite", "terrace", "terracotta", "terrapin", "terrarium", "territory", "terror", +"terrorism", "terrorist", "test", "testament", "testimonial", "testimony", "testing", "text", "textbook", "textual", "texture", "thanks", +"thaw", "theater", "theft", "theism", "theme", "theology", "theory", "therapist", "therapy", "thermals", "thermometer", "thermostat", +"thesis", "thickness", "thief", "thigh", "thing", "thinking", "thirst", "thistle", "thong", "thongs", "thorn", "thought", "thousand", +"thread", "threat", "threshold", "thrift", "thrill", "throat", "throne", "thrush", "thrust", "thug", "thumb", "thump", "thunder", "thunderbolt", +"thunderhead", "thunderstorm", "thyme", "tiara", "tic", "tick", "ticket", "tide", "tie", "tiger", "tights", "tile", "till", "tilt", "timbale", +"timber", "time", "timeline", "timeout", "timer", "timetable", "timing", "timpani", "tin", "tinderbox", "tinkle", "tintype", "tip", "tire", +"tissue", "titanium", "title", "toad", "toast", "toaster", "tobacco", "today", "toe", "toenail", "toffee", "tofu", "tog", "toga", "toilet", +"tolerance", "tolerant", "toll", "tomatillo", "tomato", "tomb", "tomography", "tomorrow", "ton", "tonality", "tone", "tongue", +"tonic", "tonight", "tool", "toot", "tooth", "toothbrush", "toothpaste", "toothpick", "top", "topic", "topsail", "toque", +"toreador", "tornado", "torso", "torte", "tortellini", "tortilla", "tortoise", "tosser", "total", "tote", "touch", "tour", +"tourism", "tourist", "tournament", "towel", "tower", "town", "townhouse", "township", "toy", "trace", "trachoma", "track", +"tracking", "tracksuit", "tract", "tractor", "trade", "trader", "trading", "tradition", "traditionalism", "traffic", "trafficker", "tragedy", +"trail", "trailer", "trailpatrol", "train", "trainer", "training", "trait", "tram", "tramp", "trance", "transaction", "transcript", "transfer", +"transformation", "transit", "transition", "translation", "transmission", "transom", "transparency", "transplantation", "transport", +"transportation", "trap", "trapdoor", "trapezium", "trapezoid", "trash", "travel", "traveler", "tray", "treasure", "treasury", "treat", +"treatment", "treaty", "tree", "trek", "trellis", "tremor", "trench", "trend", "triad", "trial", "triangle", "tribe", "tributary", "trick", +"trigger", "trigonometry", "trillion", "trim", "trinket", "trip", "tripod", "tritone", "triumph", "trolley", "trombone", "troop", "trooper", +"trophy", "trouble", "trousers", "trout", "trove", "trowel", "truck", "trumpet", "trunk", "trust", "trustee", "truth", "try", "tsunami", +"tub", "tuba", "tube", "tuber", "tug", "tugboat", "tuition", "tulip", "tumbler", "tummy", "tuna", "tune", "tunic", "tunnel", +"turban", "turf", "turkey", "turmeric", "turn", "turning", "turnip", "turnover", "turnstile", "turret", "turtle", "tusk", "tussle", "tutu", +"tuxedo", "tweet", "tweezers", "twig", "twilight", "twine", "twins", "twist", "twister", "twitter", "type", "typeface", "typewriter", +"typhoon", "ukulele", "ultimatum", "umbrella", "unblinking", "uncertainty", "uncle", "underclothes", "underestimate", "underground", +"underneath", "underpants", "underpass", "undershirt", "understanding", "understatement", "undertaker", "underwear", "underweight", "underwire", +"underwriting", "unemployment", "unibody", "uniform", "uniformity", "union", "unique", "unit", "unity", "universe", "university", "update", +"upgrade", "uplift", "upper", "upstairs", "upward", "urge", "urgency", "urn", "usage", "use", "user", "usher", "usual", "utensil", "utilisation", +"utility", "utilization", "vacation", "vaccine", "vacuum", "vagrant", "valance", "valentine", "validate", "validity", "valley", "valuable", +"value", "vampire", "van", "vanadyl", "vane", "vanilla", "vanity", "variability", "variable", "variant", "variation", "variety", "vascular", +"vase", "vault", "vaulting", "veal", "vector", "vegetable", "vegetarian", "vegetarianism", "vegetation", "vehicle", "veil", "vein", "veldt", +"vellum", "velocity", "velodrome", "velvet", "vendor", "veneer", "vengeance", "venison", "venom", "venti", "venture", "venue", "veranda", +"verb", "verdict", "verification", "vermicelli", "vernacular", "verse", "version", "vertigo", "verve", "vessel", "vest", "vestment", +"vet", "veteran", "veterinarian", "veto", "viability", "vibe", "vibraphone", "vibration", "vibrissae", "vice", "vicinity", "victim", +"victory", "video", "view", "viewer", "vignette", "villa", "village", "vine", "vinegar", "vineyard", "vintage", "vintner", "vinyl", "viola", +"violation", "violence", "violet", "violin", "virginal", "virtue", "virus", "visa", "viscose", "vise", "vision", "visit", "visitor", +"visor", "vista", "visual", "vitality", "vitamin", "vitro", "vivo", "vixen", "vodka", "vogue", "voice", "void", "vol", "volatility", +"volcano", "volleyball", "volume", "volunteer", "volunteering", "vomit", "vote", "voter", "voting", "voyage", "vulture", "wad", "wafer", +"waffle", "wage", "wagon", "waist", "waistband", "wait", "waiter", "waiting", "waitress", "waiver", "wake", "walk", "walker", "walking", +"walkway", "wall", "wallaby", "wallet", "walnut", "walrus", "wampum", "wannabe", "want", "war", "warden", "wardrobe", "warfare", "warlock", +"warlord", "warming", "warmth", "warning", "warrant", "warren", "warrior", "wasabi", "wash", "washbasin", "washcloth", "washer", +"washtub", "wasp", "waste", "wastebasket", "wasting", "watch", "watcher", "watchmaker", "water", "waterbed", "watercress", "waterfall", +"waterfront", "watermelon", "waterskiing", "waterspout", "waterwheel", "wave", "waveform", "wax", "way", "weakness", "wealth", "weapon", +"wear", "weasel", "weather", "web", "webinar", "webmail", "webpage", "website", "wedding", "wedge", "weed", "weeder", "weedkiller", "week", +"weekend", "weekender", "weight", "weird", "welcome", "welfare", "well", "west", "western", "wetland", "wetsuit", +"whack", "whale", "wharf", "wheat", "wheel", "whelp", "whey", "whip", "whirlpool", "whirlwind", "whisker", "whiskey", "whisper", "whistle", +"white", "whole", "wholesale", "wholesaler", "whorl", "wick", "widget", "widow", "width", "wife", "wifi", "wild", "wildebeest", "wilderness", +"wildlife", "will", "willingness", "willow", "win", "wind", "windage", "window", "windscreen", "windshield", "wine", "winery", +"wing", "wingman", "wingtip", "wink", "winner", "winter", "wire", "wiretap", "wiring", "wisdom", "wiseguy", "wish", "wisteria", "wit", +"witch", "withdrawal", "witness", "wok", "wolf", "woman", "wombat", "wonder", "wont", "wood", "woodchuck", "woodland", +"woodshed", "woodwind", "wool", "woolens", "word", "wording", "work", "workbench", "worker", "workforce", "workhorse", "working", "workout", +"workplace", "workshop", "world", "worm", "worry", "worship", "worshiper", "worth", "wound", "wrap", "wraparound", "wrapper", "wrapping", +"wreck", "wrecker", "wren", "wrench", "wrestler", "wriggler", "wrinkle", "wrist", "writer", "writing", "wrong", "xylophone", "yacht", +"yahoo", "yak", "yam", "yang", "yard", "yarmulke", "yarn", "yawl", "year", "yeast", "yellow", "yellowjacket", "yesterday", "yew", "yin", +"yoga", "yogurt", "yoke", "yolk", "young", "youngster", "yourself", "youth", "yoyo", "yurt", "zampone", "zebra", "zebrafish", "zen", +"zephyr", "zero", "ziggurat", "zinc", "zipper", "zither", "zombie", "zone", "zoo", "zoologist", "zoology", "zucchini" +}; + + +std::string_view obfuscateWord(std::string_view src, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func) +{ + /// Prevent using too many nouns + if (obfuscate_map.size() * 2 > nouns.size()) + throw Exception("Too many unique identifiers in queries", ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS); + + std::string_view & mapped = obfuscate_map[src]; + if (!mapped.empty()) + return mapped; + + hash_func.update(src.data(), src.size()); + std::string_view noun = nouns.begin()[hash_func.get64() % nouns.size()]; + + /// Prevent collisions + while (!used_nouns.insert(noun).second) + { + hash_func.update('\0'); + noun = nouns.begin()[hash_func.get64() % nouns.size()]; + } + + mapped = noun; + return mapped; +} + + +void obfuscateIdentifier(std::string_view src, WriteBuffer & result, WordMap & obfuscate_map, WordSet & used_nouns, SipHash hash_func) +{ + /// Find words in form 'snake_case', 'CamelCase' or 'ALL_CAPS'. + + const char * src_pos = src.data(); + const char * src_end = src_pos + src.size(); + + const char * word_begin = src_pos; + bool word_has_alphanumerics = false; + + auto append_word = [&] + { + std::string_view word(word_begin, src_pos - word_begin); + + if (keep_words.count(word)) + { + result.write(word.data(), word.size()); + } + else + { + std::string_view obfuscated_word = obfuscateWord(word, obfuscate_map, used_nouns, hash_func); + + /// Match the style of source word. + bool first_caps = !word.empty() && isUpperAlphaASCII(word[0]); + bool all_caps = first_caps && word.size() >= 2 && isUpperAlphaASCII(word[1]); + + for (size_t i = 0, size = obfuscated_word.size(); i < size; ++i) + { + if (all_caps || (i == 0 && first_caps)) + result.write(toUpperIfAlphaASCII(obfuscated_word[i])); + else + result.write(obfuscated_word[i]); + } + } + + word_begin = src_pos; + word_has_alphanumerics = false; + }; + + while (src_pos < src_end) + { + if (isAlphaNumericASCII(src_pos[0])) + word_has_alphanumerics = true; + + if (word_has_alphanumerics && src_pos[0] == '_') + { + append_word(); + result.write('_'); + ++word_begin; + } + else if (word_has_alphanumerics && isUpperAlphaASCII(src_pos[0]) && isLowerAlphaASCII(src_pos[-1])) /// xX + { + append_word(); + } + + ++src_pos; + } + + if (word_begin < src_pos) + append_word(); +} + + +void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_func) +{ + const char * src_pos = src.data(); + const char * src_end = src_pos + src.size(); + + while (src_pos < src_end) + { + /// Date + if (src_pos + strlen("0000-00-00") <= src_end + && isNumericASCII(src_pos[0]) + && isNumericASCII(src_pos[1]) + && isNumericASCII(src_pos[2]) + && isNumericASCII(src_pos[3]) + && src_pos[4] == '-' + && isNumericASCII(src_pos[5]) + && isNumericASCII(src_pos[6]) + && src_pos[7] == '-' + && isNumericASCII(src_pos[8]) + && isNumericASCII(src_pos[9])) + { + DayNum date; + ReadBufferFromMemory in(src_pos, strlen("0000-00-00")); + readDateText(date, in); + + SipHash hash_func_date = hash_func; + + if (date != 0) + { + date += hash_func_date.get64() % 256; + } + + writeDateText(date, result); + src_pos += strlen("0000-00-00"); + + /// DateTime + if (src_pos + strlen(" 00:00:00") <= src_end + && isNumericASCII(src_pos[1]) + && isNumericASCII(src_pos[2]) + && src_pos[3] == ':' + && isNumericASCII(src_pos[4]) + && isNumericASCII(src_pos[5]) + && src_pos[6] == ':' + && isNumericASCII(src_pos[7]) + && isNumericASCII(src_pos[8])) + { + result.write(src_pos[0]); + + hash_func_date.update(src_pos + 1, strlen("00:00:00")); + + uint64_t hash_value = hash_func_date.get64(); + uint32_t new_hour = hash_value % 24; + hash_value /= 24; + uint32_t new_minute = hash_value % 60; + hash_value /= 60; + uint32_t new_second = hash_value % 60; + + result.write('0' + (new_hour / 10)); + result.write('0' + (new_hour % 10)); + result.write(':'); + result.write('0' + (new_minute / 10)); + result.write('0' + (new_minute % 10)); + result.write(':'); + result.write('0' + (new_second / 10)); + result.write('0' + (new_second % 10)); + + src_pos += strlen(" 00:00:00"); + } + } + else if (isNumericASCII(src_pos[0])) + { + /// Number + if (src_pos[0] == '0' || src_pos[0] == '1') + { + /// Keep zero and one as is. + result.write(src_pos[0]); + ++src_pos; + } + else + { + ReadBufferFromMemory in(src_pos, src_end - src_pos); + uint64_t num; + readIntText(num, in); + SipHash hash_func_num = hash_func; + hash_func_num.update(src_pos, in.count()); + src_pos += in.count(); + + /// Obfuscate number but keep it within same power of two range. + + uint64_t obfuscated = hash_func_num.get64(); + uint64_t log2 = bitScanReverse(num); + + obfuscated = (1ULL << log2) + obfuscated % (1ULL << log2); + writeIntText(obfuscated, result); + } + } + else if (src_pos + 1 < src_end + && (src_pos[0] == 'e' || src_pos[0] == 'E') + && (isNumericASCII(src_pos[1]) || (src_pos[1] == '-' && src_pos + 2 < src_end && isNumericASCII(src_pos[2])))) + { + /// Something like an exponent of floating point number. Keep it as is. + /// But if it looks like a large number, overflow it into 16 bit. + + result.write(src_pos[0]); + ++src_pos; + + ReadBufferFromMemory in(src_pos, src_end - src_pos); + int16_t num; + readIntText(num, in); + writeIntText(num, result); + src_pos += in.count(); + } + else if (isAlphaASCII(src_pos[0])) + { + /// Alphabetial characters + + const char * alpha_end = src_pos + 1; + while (alpha_end < src_end && isAlphaASCII(*alpha_end)) + ++alpha_end; + + hash_func.update(src_pos, alpha_end - src_pos); + pcg64 rng(hash_func.get64()); + + while (src_pos < alpha_end) + { + auto random = rng(); + if (isLowerAlphaASCII(*src_pos)) + result.write('a' + random % 26); + else + result.write('A' + random % 26); + + ++src_pos; + } + } + else if (isASCII(src_pos[0])) + { + /// Punctuation, whitespace and control characters - keep as is. + + result.write(src_pos[0]); + ++src_pos; + } + else if (src_pos[0] <= '\xBF') + { + /// Continuation of UTF-8 sequence. + hash_func.update(src_pos[0]); + uint64_t hash = hash_func.get64(); + + char c = 0x80 + hash % (0xC0 - 0x80); + result.write(c); + + ++src_pos; + } + else if (src_pos[0]) + { + /// Start of UTF-8 sequence. + hash_func.update(src_pos[0]); + uint64_t hash = hash_func.get64(); + + if (src_pos[0] < '\xE0') + { + char c = 0xC0 + hash % 32; + result.write(c); + } + else if (src_pos[0] < '\xF0') + { + char c = 0xE0 + hash % 16; + result.write(c); + } + else + { + char c = 0xF0 + hash % 8; + result.write(c); + } + + ++src_pos; + } + } +} + +} + + +void obfuscateQueries( + std::string_view src, + WriteBuffer & result, + WordMap & obfuscate_map, + WordSet & used_nouns, + SipHash hash_func, + KnownIdentifierFunc known_identifier_func) +{ + Lexer lexer(src.data(), src.data() + src.size()); + while (true) + { + Token token = lexer.nextToken(); + std::string_view whole_token(token.begin, token.size()); + + if (token.isEnd()) + break; + + if (token.type == TokenType::BareWord) + { + if (keywords.count(whole_token) + || known_identifier_func(whole_token)) + { + /// Keep keywords as is. + result.write(token.begin, token.size()); + } + else + { + /// Obfuscate identifiers + obfuscateIdentifier(whole_token, result, obfuscate_map, used_nouns, hash_func); + } + } + else if (token.type == TokenType::QuotedIdentifier) + { + assert(token.size() >= 2); + + /// Write quotes and the obfuscated content inside. + result.write(*token.begin); + obfuscateIdentifier({token.begin + 1, token.size() - 2}, result, obfuscate_map, used_nouns, hash_func); + result.write(token.end[-1]); + } + else if (token.type == TokenType::Number) + { + obfuscateLiteral(whole_token, result, hash_func); + } + else if (token.type == TokenType::StringLiteral) + { + assert(token.size() >= 2); + + result.write(*token.begin); + obfuscateLiteral({token.begin + 1, token.size() - 2}, result, hash_func); + result.write(token.end[-1]); + } + else if (token.type == TokenType::Comment) + { + /// Skip comments - they may contain confidential info. + } + else + { + /// Everyting else is kept as is. + result.write(token.begin, token.size()); + } + } +} + +} + diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h new file mode 100644 index 00000000000..214237e19cf --- /dev/null +++ b/src/Parsers/obfuscateQueries.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +class WriteBuffer; + +using WordMap = std::unordered_map; +using WordSet = std::unordered_set; +using KnownIdentifierFunc = std::function; + +void obfuscateQueries( + std::string_view src, + WriteBuffer & result, + WordMap & obfuscate_map, + WordSet & used_nouns, + SipHash hash_func, + KnownIdentifierFunc known_identifier_func); + +} diff --git a/src/Parsers/tests/gtest_obfuscate_queries.cpp b/src/Parsers/tests/gtest_obfuscate_queries.cpp new file mode 100644 index 00000000000..ada15109d17 --- /dev/null +++ b/src/Parsers/tests/gtest_obfuscate_queries.cpp @@ -0,0 +1,92 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +using namespace DB; + + +TEST(ObfuscateQueries, Test1) +{ + WordMap obfuscated_words_map; + WordSet used_nouns; + SipHash hash_func; + + std::string salt = "Hello, world"; + hash_func.update(salt); + + SharedContextHolder shared_context; + const ContextHolder & context_holder = getContext(); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + + std::unordered_set additional_names; + + auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); + auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); + + additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); + additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); + + KnownIdentifierFunc is_known_identifier = [&](std::string_view name) + { + std::string what(name); + + return FunctionFactory::instance().tryGet(what, context_holder.context) != nullptr + || AggregateFunctionFactory::instance().isAggregateFunctionName(what) + || TableFunctionFactory::instance().isTableFunctionName(what) + || additional_names.count(what); + }; + + WriteBufferFromOwnString out; + + obfuscateQueries( + R"( +SELECT + VisitID, + Goals.ID, Goals.EventTime, + WatchIDs, + EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type +FROM merge.visits_v2 +WHERE + StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' + AND CounterID = 24226447 + AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 + AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) + AND Sign = 1 +)", + out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); + + EXPECT_EQ(out.str(), R"( +SELECT + CorduroyID, + Steel.ID, Steel.AcornSidestream, + WealthBRANCH, + GOVERNMENT.SedimentName, GOVERNMENT.SedimentExhaustion, GOVERNMENT.SedimentFencing, GOVERNMENT.SedimentOpossum, GOVERNMENT.AcornSidestream, GOVERNMENT.Lute +FROM merge.luncheonette_pants +WHERE + GovernanceCreche >= '2021-04-16' AND GovernanceCreche <= '2021-04-24' + AND StarboardID = 26446940 + AND intHash32(MessyID) = 474525514 AND intHash64(MessyID) = 13916317227779800149 + AND CorduroyID IN (5223158832904664474, 5605365157729463108, 7543250143731591192, 8715842063486405567, 7837015536326316923) + AND Tea = 1 +)"); +} + From ed4ff51ed8328c087575d4c3c4bb703356052290 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:38:59 +0300 Subject: [PATCH 108/131] Integrate into clickhouse-format; add test and comments --- programs/format/Format.cpp | 98 ++++++++++++++++--- src/Parsers/obfuscateQueries.cpp | 2 +- src/Parsers/obfuscateQueries.h | 21 ++++ .../01508_query_obfuscator.reference | 1 + .../0_stateless/01508_query_obfuscator.sh | 6 ++ 5 files changed, 113 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01508_query_obfuscator.reference create mode 100755 tests/queries/0_stateless/01508_query_obfuscator.sh diff --git a/programs/format/Format.cpp b/programs/format/Format.cpp index daf2d671568..01f952bf95e 100644 --- a/programs/format/Format.cpp +++ b/programs/format/Format.cpp @@ -1,13 +1,29 @@ #include +#include +#include #include #include #include +#include #include #include #include +#include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" @@ -22,6 +38,8 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ("oneline", "format in single line") ("quiet,q", "just check syntax, no output on success") ("multiquery,n", "allow multiple queries in the same file") + ("obfuscate", "obfuscate instead of formatting") + ("seed", po::value(), "seed (arbitrary string) that determines the result of obfuscation") ; boost::program_options::variables_map options; @@ -40,10 +58,17 @@ int mainEntryClickHouseFormat(int argc, char ** argv) bool oneline = options.count("oneline"); bool quiet = options.count("quiet"); bool multiple = options.count("multiquery"); + bool obfuscate = options.count("obfuscate"); - if (quiet && (hilite || oneline)) + if (quiet && (hilite || oneline || obfuscate)) { - std::cerr << "Options 'hilite' or 'oneline' have no sense in 'quiet' mode." << std::endl; + std::cerr << "Options 'hilite' or 'oneline' or 'obfuscate' have no sense in 'quiet' mode." << std::endl; + return 2; + } + + if (obfuscate && (hilite || oneline || quiet)) + { + std::cerr << "Options 'hilite' or 'oneline' or 'quiet' have no sense in 'obfuscate' mode." << std::endl; return 2; } @@ -51,21 +76,66 @@ int mainEntryClickHouseFormat(int argc, char ** argv) ReadBufferFromFileDescriptor in(STDIN_FILENO); readStringUntilEOF(query, in); - const char * pos = query.data(); - const char * end = pos + query.size(); - - ParserQuery parser(end); - do + if (obfuscate) { - ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - if (!quiet) + WordMap obfuscated_words_map; + WordSet used_nouns; + SipHash hash_func; + + if (options.count("seed")) { - formatAST(*res, std::cout, hilite, oneline); - if (multiple) - std::cout << "\n;\n"; - std::cout << std::endl; + std::string seed; + hash_func.update(options["seed"].as()); } - } while (multiple && pos != end); + + SharedContextHolder shared_context = Context::createShared(); + Context context = Context::createGlobal(shared_context.get()); + context.makeGlobalContext(); + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + + std::unordered_set additional_names; + + auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); + auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); + + additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); + additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); + + KnownIdentifierFunc is_known_identifier = [&](std::string_view name) + { + std::string what(name); + + return FunctionFactory::instance().tryGet(what, context) != nullptr + || AggregateFunctionFactory::instance().isAggregateFunctionName(what) + || TableFunctionFactory::instance().isTableFunctionName(what) + || additional_names.count(what); + }; + + WriteBufferFromFileDescriptor out(STDOUT_FILENO); + obfuscateQueries(query, out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); + } + else + { + const char * pos = query.data(); + const char * end = pos + query.size(); + + ParserQuery parser(end); + do + { + ASTPtr res = parseQueryAndMovePosition(parser, pos, end, "query", multiple, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + if (!quiet) + { + formatAST(*res, std::cout, hilite, oneline); + if (multiple) + std::cout << "\n;\n"; + std::cout << std::endl; + } + } while (multiple && pos != end); + } } catch (...) { diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 66221005d77..9dcf3d6ada3 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -836,7 +836,7 @@ void obfuscateLiteral(std::string_view src, WriteBuffer & result, SipHash hash_f ++src_pos; } - else if (src_pos[0]) + else { /// Start of UTF-8 sequence. hash_func.update(src_pos[0]); diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h index 214237e19cf..2e65ec427f6 100644 --- a/src/Parsers/obfuscateQueries.h +++ b/src/Parsers/obfuscateQueries.h @@ -18,6 +18,27 @@ using WordMap = std::unordered_map; using WordSet = std::unordered_set; using KnownIdentifierFunc = std::function; +/** Takes one or multiple queries and obfuscate them by replacing identifiers to pseudorandom words + * and replacing literals to random values, while preserving the structure of the queries and the general sense. + * + * Its intended use case is when the user wants to share their queries for testing and debugging + * but is afraid to disclose the details about their column names, domain area and values of constants. + * + * It can obfuscate multiple queries in consistent fashion - identical names will be transformed to identical results. + * + * The function is not guaranteed to always give correct result or to be secure. It's implemented in "best effort" fashion. + * + * @param src - a string with source queries. + * @param result - where the obfuscated queries will be written. + * @param obfuscate_map - information about substituted identifiers + * (pass empty map at the beginning or reuse it from previous invocation to get consistent result) + * @param used_nouns - information about words used for substitution + * (pass empty set at the beginning or reuse it from previous invocation to get consistent result) + * @param hash_func - hash function that will be used as a pseudorandom source, + * it's recommended to preseed the function before passing here. + * @param known_identifier_func - a function that returns true if identifier is known name + * (of function, aggregate function, etc. that should be kept as is). If it returns false, identifier will be obfuscated. + */ void obfuscateQueries( std::string_view src, WriteBuffer & result, diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference new file mode 100644 index 00000000000..e2f61cf345d --- /dev/null +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -0,0 +1 @@ +SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Meantime_id_brisketHeavy, exclamation(Grit), avgIf(remote('128.0.0.1')) diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh new file mode 100755 index 00000000000..797271edb30 --- /dev/null +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" From 673f72cf8c147fa0252052e87ae0b2346d2e0e6f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:48:09 +0300 Subject: [PATCH 109/131] Remove some profanity --- src/Parsers/obfuscateQueries.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 9dcf3d6ada3..e8ea68ba0c6 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -76,7 +76,7 @@ std::initializer_list nouns "architect", "architecture", "archives", "area", "arena", "argument", "arithmetic", "ark", "arm", "armadillo", "armament", "armchair", "armoire", "armor", "armour", "armpit", "armrest", "army", "arrangement", "array", "arrest", "arrival", "arrogance", "arrow", "art", "artery", "arthur", "artichoke", "article", "artifact", "artificer", "artist", "ascend", "ascent", "ascot", "ash", "ashram", "ashtray", -"aside", "asparagus", "aspect", "asphalt", "aspic", "ass", "assassination", "assault", "assembly", "assertion", "assessment", "asset", +"aside", "asparagus", "aspect", "asphalt", "aspic", "assassination", "assault", "assembly", "assertion", "assessment", "asset", "assignment", "assist", "assistance", "assistant", "associate", "association", "assumption", "assurance", "asterisk", "astrakhan", "astrolabe", "astrologer", "astrology", "astronomy", "asymmetry", "atelier", "atheist", "athlete", "athletics", "atmosphere", "atom", "atrium", "attachment", "attack", "attacker", "attainment", "attempt", "attendance", "attendant", "attention", "attenuation", "attic", "attitude", "attorney", @@ -182,7 +182,7 @@ std::initializer_list nouns "current", "curriculum", "curry", "curse", "cursor", "curtailment", "curtain", "curve", "cushion", "custard", "custody", "custom", "customer", "cut", "cuticle", "cutlet", "cutover", "cutting", "cyclamen", "cycle", "cyclone", "cyclooxygenase", "cygnet", "cylinder", "cymbal", "cynic", "cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", -"dame", "damn", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", +"dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", "data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", "death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", "deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", @@ -193,8 +193,7 @@ std::initializer_list nouns "description", "desert", "design", "designation", "designer", "desire", "desk", "desktop", "dessert", "destination", "destiny", "destroyer", "destruction", "detail", "detainee", "detainment", "detection", "detective", "detector", "detention", "determination", "detour", "devastation", "developer", "developing", "development", "developmental", "deviance", "deviation", "device", "devil", "dew", "dhow", "diabetes", "diadem", -"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dick", -"dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", +"diagnosis", "diagram", "dial", "dialect", "dialogue", "diam", "diamond", "diaper", "diaphragm", "diarist", "diary", "dibble", "dickey", "dictaphone", "dictator", "diction", "dictionary", "die", "diesel", "diet", "difference", "differential", "difficulty", "diffuse", "dig", "digestion", "digestive", "digger", "digging", "digit", "dignity", "dilapidation", "dill", "dilution", "dime", "dimension", "dimple", "diner", "dinghy", "dining", "dinner", "dinosaur", "dioxide", "dip", "diploma", "diplomacy", "dipstick", "direction", "directive", "director", "directory", "dirndl", "dirt", "disability", "disadvantage", "disagreement", "disappointment", "disarmament", "disaster", "discharge", @@ -254,7 +253,7 @@ std::initializer_list nouns "fortress", "fortune", "forum", "foundation", "founder", "founding", "fountain", "fourths", "fowl", "fox", "foxglove", "fraction", "fragrance", "frame", "framework", "fratricide", "fraud", "fraudster", "freak", "freckle", "freedom", "freelance", "freezer", "freezing", "freight", "freighter", "frenzy", "freon", "frequency", "fresco", "friction", "fridge", "friend", "friendship", "fries", "frigate", "fright", "fringe", -"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuck", "fuel", "fugato", +"fritter", "frock", "frog", "front", "frontier", "frost", "frosting", "frown", "fruit", "frustration", "fry", "fuel", "fugato", "fulfillment", "full", "fun", "function", "functionality", "fund", "funding", "fundraising", "funeral", "fur", "furnace", "furniture", "furry", "fusarium", "futon", "future", "gadget", "gaffe", "gaffer", "gain", "gaiters", "gale", "gallery", "galley", "gallon", "galoshes", "gambling", "game", "gamebird", "gaming", "gander", "gang", "gap", "garage", "garb", "garbage", "garden", @@ -413,7 +412,7 @@ std::initializer_list nouns "picture", "picturesque", "pie", "piece", "pier", "piety", "pig", "pigeon", "piglet", "pigpen", "pigsty", "pike", "pilaf", "pile", "pilgrim", "pilgrimage", "pill", "pillar", "pillbox", "pillow", "pilot", "pimp", "pimple", "pin", "pinafore", "pine", "pineapple", "pinecone", "ping", "pink", "pinkie", "pinot", "pinstripe", "pint", "pinto", "pinworm", "pioneer", "pipe", "pipeline", "piracy", "pirate", -"piss", "pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", +"pistol", "pit", "pita", "pitch", "pitcher", "pitching", "pith", "pizza", "place", "placebo", "placement", "placode", "plagiarism", "plain", "plaintiff", "plan", "plane", "planet", "planning", "plant", "plantation", "planter", "planula", "plaster", "plasterboard", "plastic", "plate", "platelet", "platform", "platinum", "platter", "platypus", "play", "player", "playground", "playroom", "playwright", "plea", "pleasure", "pleat", "pledge", "plenty", "plier", "pliers", "plight", "plot", "plough", "plover", "plow", "plowman", "plug", @@ -466,7 +465,7 @@ std::initializer_list nouns "riot", "rip", "ripple", "rise", "riser", "risk", "rite", "ritual", "river", "riverbed", "rivulet", "road", "roadway", "roar", "roast", "robe", "robin", "robot", "robotics", "rock", "rocker", "rocket", "rod", "role", "roll", "roller", "romaine", "romance", "roof", "room", "roommate", "rooster", "root", "rope", "rose", "rosemary", "roster", "rostrum", "rotation", "round", "roundabout", "route", -"router", "routine", "row", "rowboat", "rowing", "rubber", "rubbish", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", +"router", "routine", "row", "rowboat", "rowing", "rubber", "rubric", "ruby", "ruckus", "rudiment", "ruffle", "rug", "rugby", "ruin", "rule", "ruler", "ruling", "rum", "rumor", "run", "runaway", "runner", "running", "runway", "rush", "rust", "rutabaga", "rye", "sabre", "sac", "sack", "saddle", "sadness", "safari", "safe", "safeguard", "safety", "saffron", "sage", "sail", "sailboat", "sailing", "sailor", "saint", "sake", "salad", "salami", "salary", "sale", "salesman", "salmon", "salon", "saloon", "salsa", "salt", "salute", "samovar", @@ -486,7 +485,7 @@ std::initializer_list nouns "setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", "shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", "shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", -"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shit", "shoat", "shock", "shoe", "shoe-horn", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoe-horn", "shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", "short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", "shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", @@ -494,7 +493,7 @@ std::initializer_list nouns "silicon", "silk", "silkworm", "sill", "silly", "silo", "silver", "similarity", "simple", "simplicity", "simplification", "simvastatin", "sin", "singer", "singing", "singular", "sink", "sinuosity", "sip", "sir", "sister", "sitar", "site", "situation", "size", "skate", "skating", "skean", "skeleton", "ski", "skiing", "skill", "skin", "skirt", "skull", "skullcap", "skullduggery", "skunk", "sky", -"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slave", "slavery", "slaw", "sled", "sledge", +"skylight", "skyline", "skyscraper", "skywalk", "slang", "slapstick", "slash", "slate", "slavery", "slaw", "sled", "sledge", "sleep", "sleepiness", "sleeping", "sleet", "sleuth", "slice", "slide", "slider", "slime", "slip", "slipper", "slippers", "slope", "slot", "sloth", "slump", "smell", "smelting", "smile", "smith", "smock", "smog", "smoke", "smoking", "smolt", "smuggling", "snack", "snail", "snake", "snakebite", "snap", "snarl", "sneaker", "sneakers", "sneeze", "sniffle", "snob", "snorer", "snow", "snowboarding", "snowflake", From 22861b0c3dcd864584954b79436000a38306d130 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:49:30 +0300 Subject: [PATCH 110/131] Remove some profanity --- src/Parsers/obfuscateQueries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index e8ea68ba0c6..03f0cef2605 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -171,7 +171,7 @@ std::initializer_list nouns "councilperson", "counsel", "counseling", "counselling", "counsellor", "counselor", "count", "counter", "counterpart", "counterterrorism", "countess", "country", "countryside", "county", "couple", "coupon", "courage", "course", "court", "courthouse", "courtroom", "cousin", "covariate", "cover", "coverage", "coverall", "cow", "cowbell", "cowboy", "coyote", "crab", "crack", "cracker", "crackers", -"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crap", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", +"cradle", "craft", "craftsman", "cranberry", "crane", "cranky", "crash", "crate", "cravat", "craw", "crawdad", "crayfish", "crayon", "crazy", "cream", "creation", "creationism", "creationist", "creative", "creativity", "creator", "creature", "creche", "credential", "credenza", "credibility", "credit", "creditor", "creek", "creme brulee", "crepe", "crest", "crew", "crewman", "crewmate", "crewmember", "crewmen", "cria", "crib", "cribbage", "cricket", "cricketer", "crime", "criminal", "crinoline", "crisis", "crisp", "criteria", "criterion", From d78fc63deb0d8a37e7a10e045b5807e9dd154ca7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:50:00 +0300 Subject: [PATCH 111/131] Remove duplicate word --- src/Parsers/obfuscateQueries.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 03f0cef2605..17e751f5149 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -450,7 +450,7 @@ std::initializer_list nouns "recipient", "reciprocity", "reclamation", "recliner", "recognition", "recollection", "recommendation", "reconsideration", "record", "recorder", "recording", "recovery", "recreation", "recruit", "rectangle", "red", "redesign", "redhead", "redirect", "rediscovery", "reduction", "reef", "refectory", "reference", "referendum", "reflection", "reform", "refreshments", "refrigerator", "refuge", "refund", "refusal", -"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalisation", +"refuse", "regard", "regime", "region", "regionalism", "register", "registration", "registry", "regret", "regulation", "regulator", "rehospitalization", "reindeer", "reinscription", "reject", "relation", "relationship", "relative", "relaxation", "relay", "release", "reliability", "relief", "religion", "relish", "reluctance", "remains", "remark", "reminder", "remnant", "remote", "removal", "renaissance", "rent", "reorganisation", "reorganization", "repair", "reparation", "repayment", "repeat", "replacement", "replica", "replication", "reply", From fc10803deffd4585bb516753552a8c1eb387f246 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:50:56 +0300 Subject: [PATCH 112/131] Fix comment --- src/Parsers/obfuscateQueries.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.h b/src/Parsers/obfuscateQueries.h index 2e65ec427f6..0a192649a92 100644 --- a/src/Parsers/obfuscateQueries.h +++ b/src/Parsers/obfuscateQueries.h @@ -18,7 +18,7 @@ using WordMap = std::unordered_map; using WordSet = std::unordered_set; using KnownIdentifierFunc = std::function; -/** Takes one or multiple queries and obfuscate them by replacing identifiers to pseudorandom words +/** Takes one or multiple queries and obfuscates them by replacing identifiers to pseudorandom words * and replacing literals to random values, while preserving the structure of the queries and the general sense. * * Its intended use case is when the user wants to share their queries for testing and debugging From bda2c2f58fcc590bed044200f4a5e0c563691d54 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 03:53:26 +0300 Subject: [PATCH 113/131] Remove sensitive words --- src/Parsers/obfuscateQueries.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 17e751f5149..0d62fe435d8 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -283,10 +283,10 @@ std::initializer_list nouns "hearsay", "heart", "heartache", "heartbeat", "hearth", "hearthside", "heartwood", "heat", "heater", "heating", "heaven", "heavy", "hectare", "hedge", "hedgehog", "heel", "heifer", "height", "heir", "heirloom", "helicopter", "helium", "hell", "hellcat", "hello", "helmet", "helo", "help", "hemisphere", "hemp", "hen", "hepatitis", "herb", "herbs", "heritage", "hermit", "hero", "heroine", "heron", -"herring", "hesitation", "heterosexual", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", +"herring", "hesitation", "hexagon", "heyday", "hiccups", "hide", "hierarchy", "high", "highland", "highlight", "highway", "hike", "hiking", "hill", "hint", "hip", "hippodrome", "hippopotamus", "hire", "hiring", "historian", "history", "hit", "hive", "hobbit", "hobby", "hockey", "hoe", "hog", "hold", "holder", "hole", "holiday", "home", "homeland", "homeownership", "hometown", "homework", -"homicide", "homogenate", "homonym", "homosexual", "homosexuality", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", +"homicide", "homogenate", "homonym", "honesty", "honey", "honeybee", "honeydew", "honor", "honoree", "hood", "hoof", "hook", "hop", "hope", "hops", "horde", "horizon", "hormone", "horn", "hornet", "horror", "horse", "horseradish", "horst", "hose", "hosiery", "hospice", "hospital", "hospitalisation", "hospitality", "hospitalization", "host", "hostel", "hostess", "hotdog", "hotel", "hound", "hour", "hourglass", "house", "houseboat", "household", "housewife", "housework", "housing", "hovel", "hovercraft", "howard", @@ -328,7 +328,7 @@ std::initializer_list nouns "laughter", "laundry", "lava", "law", "lawmaker", "lawn", "lawsuit", "lawyer", "lay", "layer", "layout", "lead", "leader", "leadership", "leading", "leaf", "league", "leaker", "leap", "learning", "leash", "leather", "leave", "leaver", "lecture", "leek", "leeway", "left", "leg", "legacy", "legal", "legend", "legging", "legislation", "legislator", "legislature", "legitimacy", "legume", "leisure", "lemon", -"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesbian", "lesson", "letter", +"lemonade", "lemur", "lender", "lending", "length", "lens", "lentil", "leopard", "leprosy", "leptocephalus", "lesson", "letter", "lettuce", "level", "lever", "leverage", "leveret", "liability", "liar", "liberty", "libido", "library", "licence", "license", "licensing", "licorice", "lid", "lie", "lieu", "lieutenant", "life", "lifestyle", "lifetime", "lift", "ligand", "light", "lighting", "lightning", "lightscreen", "ligula", "likelihood", "likeness", "lilac", "lily", "limb", "lime", "limestone", "limit", "limitation", "limo", "line", From f97d40584d0011644e0967a0ba8bd5c9cc595e93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 04:10:26 +0300 Subject: [PATCH 114/131] Fix "Arcadia" --- src/Parsers/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ya.make b/src/Parsers/ya.make index 0a0c301b722..4ec97b8b55b 100644 --- a/src/Parsers/ya.make +++ b/src/Parsers/ya.make @@ -85,6 +85,7 @@ SRCS( MySQL/ASTDeclareReference.cpp MySQL/ASTDeclareSubPartition.cpp MySQL/ASTDeclareTableOptions.cpp + obfuscateQueries.cpp parseDatabaseAndTableName.cpp parseIdentifierOrStringLiteral.cpp parseIntervalKind.cpp From 4445cb649525488ba5dec9180a36f58bef547d8c Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 25 Sep 2020 22:20:36 -0300 Subject: [PATCH 115/131] tests for partition pruning --- .../01508_partition_pruning.queries | 118 ++++++++++++ .../01508_partition_pruning.reference | 177 ++++++++++++++++++ .../0_stateless/01508_partition_pruning.sh | 37 ++++ 3 files changed, 332 insertions(+) create mode 100644 tests/queries/0_stateless/01508_partition_pruning.queries create mode 100644 tests/queries/0_stateless/01508_partition_pruning.reference create mode 100755 tests/queries/0_stateless/01508_partition_pruning.sh diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries new file mode 100644 index 00000000000..05f46e8e496 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -0,0 +1,118 @@ +DROP TABLE IF EXISTS tMM +DROP TABLE IF EXISTS tDD +DROP TABLE IF EXISTS sDD +DROP TABLE IF EXISTS xMM +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() +SYSTEM STOP MERGES tMM; +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) + +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); + +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); +insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); + +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() +SYSTEM STOP MERGES xMM; +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000) + + + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' + +SYSTEM START MERGES tMM +OPTIMIZE TABLE tMM FINAL + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 + +OPTIMIZE TABLE tDD FINAL + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' + + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +select uniqExact(_part), count() from sDD where d >= 1598918400000 +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 + + + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 66 +select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a = 2 + +SYSTEM START MERGES xMM; +optimize table xMM final; + +select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 + +DROP TABLE tMM +DROP TABLE tDD +DROP TABLE sDD +DROP TABLE xMM + + + diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference new file mode 100644 index 00000000000..d967c760e91 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -0,0 +1,177 @@ +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges + +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000 +Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges + +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1 +Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges + +select uniqExact(_part), count() from xMM where a = 66 +Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges + +select uniqExact(_part), count() from xMM where a <> 66 +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + +select uniqExact(_part), count() from xMM where a = 2 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where a = 1 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + +select uniqExact(_part), count() from xMM where a <> 66 +Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges + +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges + diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh new file mode 100755 index 00000000000..3d5f58ca73d --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +#-------------------------------------------- +# Description of test result: +# Test the correctness of the optimization +# by asserting read marks in the log. +# Relation of read marks and optimization: +# read marks = +# the number of monotonic marks filtered through predicates +# + no monotonic marks count +#-------------------------------------------- + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh + +#export CLICKHOUSE_CLIENT="clickhouse-client --send_logs_level=none" +#export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none +#export CURDIR=. + + +queries="${CURDIR}/01508_partition_pruning.queries" +while IFS= read -r sql +do + [ -z "$sql" ] && continue + if [[ "$sql" == select* ]] ; + then + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') + echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') + echo "" + else + ${CLICKHOUSE_CLIENT} --query "$sql" + fi +done < "$queries" + + From 6f1a144f199a347ea7edaab584ff964029712dc1 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Fri, 25 Sep 2020 22:23:29 -0300 Subject: [PATCH 116/131] tests for partition pruning --- tests/queries/0_stateless/01508_partition_pruning.sh | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index 3d5f58ca73d..e06b907f19f 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -2,12 +2,8 @@ #-------------------------------------------- # Description of test result: -# Test the correctness of the optimization -# by asserting read marks in the log. -# Relation of read marks and optimization: -# read marks = -# the number of monotonic marks filtered through predicates -# + no monotonic marks count +# Test the correctness of the partition +# pruning #-------------------------------------------- CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 3a2c809173c526c93ea27c7796e1d8cbbdbe8586 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 04:59:25 +0300 Subject: [PATCH 117/131] Add missing dependencies --- programs/format/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/programs/format/CMakeLists.txt b/programs/format/CMakeLists.txt index ab06708cd3a..49f17ef163f 100644 --- a/programs/format/CMakeLists.txt +++ b/programs/format/CMakeLists.txt @@ -5,6 +5,9 @@ set (CLICKHOUSE_FORMAT_LINK boost::program_options clickhouse_common_io clickhouse_parsers + clickhouse_functions + clickhouse_aggregate_functions + clickhouse_table_functions dbms ) From c51502bce47b7d6fbcc0ab696d82d115a43b584c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:06:14 +0300 Subject: [PATCH 118/131] Remove words with dashes --- src/Parsers/obfuscateQueries.cpp | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 0d62fe435d8..fa7c5b4d3e4 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -103,18 +103,18 @@ std::initializer_list nouns "blowhole", "blue", "blueberry", "blush", "boar", "board", "boat", "boatload", "boatyard", "bob", "bobcat", "body", "bog", "bolero", "bolt", "bomb", "bomber", "bombing", "bond", "bonding", "bondsman", "bone", "bonfire", "bongo", "bonnet", "bonsai", "bonus", "boogeyman", "book", "bookcase", "bookend", "booking", "booklet", "bookmark", "boolean", "boom", "boon", "boost", "booster", "boot", "bootee", "bootie", -"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "bottom-line", +"booty", "border", "bore", "borrower", "borrowing", "bosom", "boss", "botany", "bother", "bottle", "bottling", "bottom", "boudoir", "bough", "boulder", "boulevard", "boundary", "bouquet", "bourgeoisie", "bout", "boutique", "bow", "bower", "bowl", "bowler", "bowling", "bowtie", "box", "boxer", "boxspring", "boy", "boycott", "boyfriend", "boyhood", "boysenberry", "bra", "brace", "bracelet", "bracket", "brain", "brake", "bran", "branch", "brand", "brandy", "brass", "brassiere", "bratwurst", "bread", "breadcrumb", "breadfruit", "break", "breakdown", "breakfast", "breakpoint", "breakthrough", "breast", "breastplate", "breath", "breeze", "brewer", "bribery", "brick", "bricklaying", "bride", "bridge", "brief", "briefing", "briefly", "briefs", "brilliant", "brink", "brisket", "broad", "broadcast", "broccoli", -"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brother-in-law", +"brochure", "brocolli", "broiler", "broker", "bronchitis", "bronco", "bronze", "brooch", "brood", "brook", "broom", "brother", "brow", "brown", "brownie", "browser", "browsing", "brunch", "brush", "brushfire", "brushing", "bubble", "buck", "bucket", "buckle", "buckwheat", "bud", "buddy", "budget", "buffalo", "buffer", "buffet", "bug", "buggy", "bugle", "builder", "building", "bulb", "bulk", "bull", "bulldozer", "bullet", "bump", "bumper", "bun", "bunch", "bungalow", "bunghole", "bunkhouse", "burden", "bureau", "burglar", "burial", "burlesque", "burn", "burning", "burrito", "burro", "burrow", "burst", "bus", "bush", "business", "businessman", -"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "c-clamp", +"bust", "bustle", "butane", "butcher", "butler", "butter", "butterfly", "button", "buy", "buyer", "buying", "buzz", "buzzard", "cabana", "cabbage", "cabin", "cabinet", "cable", "caboose", "cacao", "cactus", "caddy", "cadet", "cafe", "caffeine", "caftan", "cage", "cake", "calcification", "calculation", "calculator", "calculus", "calendar", "calf", "caliber", "calibre", "calico", "call", "calm", "calorie", "camel", "cameo", "camera", "camp", "campaign", "campaigning", "campanile", "camper", "campus", "can", "canal", "cancer", @@ -184,7 +184,7 @@ std::initializer_list nouns "cyst", "cytokine", "cytoplasm", "dad", "daddy", "daffodil", "dagger", "dahlia", "daikon", "daily", "dairy", "daisy", "dam", "damage", "dame", "dance", "dancer", "dancing", "dandelion", "danger", "dare", "dark", "darkness", "darn", "dart", "dash", "dashboard", "data", "database", "date", "daughter", "dawn", "day", "daybed", "daylight", "dead", "deadline", "deal", "dealer", "dealing", "dearest", -"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "decision-making", +"death", "deathwatch", "debate", "debris", "debt", "debtor", "decade", "decadence", "decency", "decimal", "decision", "deck", "declaration", "declination", "decline", "decoder", "decongestant", "decoration", "decrease", "decryption", "dedication", "deduce", "deduction", "deed", "deep", "deer", "default", "defeat", "defendant", "defender", "defense", "deficit", "definition", "deformation", "degradation", "degree", "delay", "deliberation", "delight", "delivery", "demand", "democracy", "democrat", "demon", "demur", "den", @@ -210,7 +210,7 @@ std::initializer_list nouns "dress", "dresser", "dressing", "drill", "drink", "drinking", "drive", "driver", "driveway", "driving", "drizzle", "dromedary", "drop", "drudgery", "drug", "drum", "drummer", "drunk", "dryer", "duck", "duckling", "dud", "dude", "due", "duel", "dueling", "duffel", "dugout", "dulcimer", "dumbwaiter", "dump", "dump truck", "dune", "dune buggy", "dungarees", "dungeon", "duplexer", "duration", "durian", "dusk", -"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "e-book", +"dust", "dust storm", "duster", "duty", "dwarf", "dwell", "dwelling", "dynamics", "dynamite", "dynamo", "dynasty", "dysfunction", "eagle", "eaglet", "ear", "eardrum", "earmuffs", "earnings", "earplug", "earring", "earrings", "earth", "earthquake", "earthworm", "ease", "easel", "east", "eating", "eaves", "eavesdropper", "ecclesia", "echidna", "eclipse", "ecliptic", "ecology", "economics", "economy", "ecosystem", "ectoderm", "ectodermal", "ecumenist", "eddy", "edge", "edger", "edible", "editing", "edition", "editor", "editorial", @@ -273,7 +273,7 @@ std::initializer_list nouns "grill", "grin", "grip", "gripper", "grit", "grocery", "ground", "group", "grouper", "grouse", "grove", "growth", "grub", "guacamole", "guarantee", "guard", "guava", "guerrilla", "guess", "guest", "guestbook", "guidance", "guide", "guideline", "guilder", "guilt", "guilty", "guinea", "guitar", "guitarist", "gum", "gumshoe", "gun", "gunpowder", "gutter", "guy", "gym", "gymnast", "gymnastics", "gynaecology", -"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "half-sister", +"gyro", "habit", "habitat", "hacienda", "hacksaw", "hackwork", "hail", "hair", "haircut", "hake", "half", "halibut", "hall", "halloween", "hallway", "halt", "ham", "hamburger", "hammer", "hammock", "hamster", "hand", "handball", "handful", "handgun", "handicap", "handle", "handlebar", "handmaiden", "handover", "handrail", "handsaw", "hanger", "happening", "happiness", "harald", "harbor", "harbour", "hardboard", "hardcover", "hardening", "hardhat", "hardship", "hardware", "hare", "harm", @@ -293,7 +293,7 @@ std::initializer_list nouns "howitzer", "hub", "hubcap", "hubris", "hug", "hugger", "hull", "human", "humanity", "humidity", "hummus", "humor", "humour", "hunchback", "hundred", "hunger", "hunt", "hunter", "hunting", "hurdle", "hurdler", "hurricane", "hurry", "hurt", "husband", "hut", "hutch", "hyacinth", "hybridisation", "hybridization", "hydrant", "hydraulics", "hydrocarb", "hydrocarbon", "hydrofoil", "hydrogen", "hydrolyse", "hydrolysis", -"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "ice-cream", +"hydrolyze", "hydroxyl", "hyena", "hygienic", "hype", "hyphenation", "hypochondria", "hypothermia", "hypothesis", "ice", "iceberg", "icebreaker", "icecream", "icicle", "icing", "icon", "icy", "id", "idea", "ideal", "identification", "identity", "ideology", "idiom", "idiot", "igloo", "ignorance", "ignorant", "ikebana", "illegal", "illiteracy", "illness", "illusion", "illustration", "image", "imagination", "imbalance", "imitation", "immigrant", "immigration", "immortal", "impact", "impairment", "impala", "impediment", "implement", @@ -320,7 +320,7 @@ std::initializer_list nouns "kamikaze", "kangaroo", "karate", "kayak", "kazoo", "kebab", "keep", "keeper", "kendo", "kennel", "ketch", "ketchup", "kettle", "kettledrum", "key", "keyboard", "keyboarding", "keystone", "kick", "kid", "kidney", "kielbasa", "kill", "killer", "killing", "kilogram", "kilometer", "kilt", "kimono", "kinase", "kind", "kindness", "king", "kingdom", "kingfish", "kiosk", "kiss", "kit", "kitchen", "kite", -"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "know-how", +"kitsch", "kitten", "kitty", "kiwi", "knee", "kneejerk", "knickers", "knife", "knight", "knitting", "knock", "knot", "knowledge", "knuckle", "koala", "kohlrabi", "kumquat", "lab", "label", "labor", "laboratory", "laborer", "labour", "labourer", "lace", "lack", "lacquerware", "lad", "ladder", "ladle", "lady", "ladybug", "lag", "lake", "lamb", "lambkin", "lament", "lamp", "lanai", "land", "landform", "landing", "landmine", "landscape", "lane", "language", "lantern", "lap", "laparoscope", "lapdog", "laptop", "larch", "lard", @@ -401,7 +401,7 @@ std::initializer_list nouns "paste", "pastor", "pastoralist", "pastry", "pasture", "pat", "patch", "pate", "patent", "patentee", "path", "pathogenesis", "pathology", "pathway", "patience", "patient", "patina", "patio", "patriarch", "patrimony", "patriot", "patrol", "patroller", "patrolling", "patron", "pattern", "patty", "pattypan", "pause", "pavement", "pavilion", "paw", "pawnshop", "pay", "payee", "payment", "payoff", "pea", "peace", -"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "peer-to-peer", +"peach", "peacoat", "peacock", "peak", "peanut", "pear", "pearl", "peasant", "pecan", "pecker", "pedal", "peek", "peen", "peer", "pegboard", "pelican", "pelt", "pen", "penalty", "pence", "pencil", "pendant", "pendulum", "penguin", "penicillin", "peninsula", "penis", "pennant", "penny", "pension", "pentagon", "peony", "people", "pepper", "pepperoni", "percent", "percentage", "perception", "perch", "perennial", "perfection", "performance", "perfume", "period", "periodical", "peripheral", "permafrost", "permission", "permit", "perp", @@ -485,7 +485,7 @@ std::initializer_list nouns "setback", "setting", "settlement", "settler", "severity", "sewer", "sex", "sexuality", "shack", "shackle", "shade", "shadow", "shadowbox", "shakedown", "shaker", "shallot", "shallows", "shame", "shampoo", "shanty", "shape", "share", "shareholder", "shark", "shaw", "shawl", "shear", "shearling", "sheath", "shed", "sheep", "sheet", "shelf", "shell", "shelter", "sherbet", "sherry", "shield", "shift", "shin", -"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoe-horn", +"shine", "shingle", "ship", "shipper", "shipping", "shipyard", "shirt", "shirtdress", "shoat", "shock", "shoe", "shoehorn", "shoelace", "shoemaker", "shoes", "shoestring", "shofar", "shoot", "shootdown", "shop", "shopper", "shopping", "shore", "shoreline", "short", "shortage", "shorts", "shortwave", "shot", "shoulder", "shout", "shovel", "show", "shower", "shred", "shrimp", "shrine", "shutdown", "sibling", "sick", "sickness", "side", "sideboard", "sideburns", "sidecar", "sidestream", "sidewalk", "siding", @@ -513,7 +513,7 @@ std::initializer_list nouns "staff", "stag", "stage", "stain", "stair", "staircase", "stake", "stalk", "stall", "stallion", "stamen", "stamina", "stamp", "stance", "stand", "standard", "standardisation", "standardization", "standing", "standoff", "standpoint", "star", "starboard", "start", "starter", "state", "statement", "statin", "station", "statistic", "statistics", "statue", "status", "statute", "stay", "steak", -"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "step-daughter", +"stealth", "steam", "steamroller", "steel", "steeple", "stem", "stench", "stencil", "step", "stepdaughter", "stepmother", "stepson", "stereo", "stew", "steward", "stick", "sticker", "stiletto", "still", "stimulation", "stimulus", "sting", "stinger", "stitch", "stitcher", "stock", "stockings", "stole", "stomach", "stone", "stonework", "stool", @@ -533,7 +533,7 @@ std::initializer_list nouns "swivel", "sword", "swordfight", "swordfish", "sycamore", "symbol", "symmetry", "sympathy", "symptom", "syndicate", "syndrome", "synergy", "synod", "synonym", "synthesis", "syrup", "system", "tab", "tabby", "tabernacle", "table", "tablecloth", "tablet", "tabletop", "tachometer", "tackle", "taco", "tactics", "tactile", "tadpole", "tag", "tail", "tailbud", "tailor", "tailspin", "takeover", -"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tank-top", +"tale", "talent", "talk", "talking", "tamale", "tambour", "tambourine", "tan", "tandem", "tangerine", "tank", "tanker", "tankful", "tap", "tape", "tapioca", "target", "taro", "tarragon", "tart", "task", "tassel", "taste", "tatami", "tattler", "tattoo", "tavern", "tax", "taxi", "taxicab", "taxpayer", "tea", "teacher", "teaching", "team", "teammate", "teapot", "tear", "tech", "technician", "technique", "technologist", "technology", "tectonics", "teen", "teenager", "teepee", "telephone", "telescreen", "teletype", From f3349c8d138e882793e01f8708b7e0fadb7ad937 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:13:20 +0300 Subject: [PATCH 119/131] Keywords are case-insensitive --- src/Parsers/obfuscateQueries.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index fa7c5b4d3e4..3594bdef3eb 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -884,7 +885,10 @@ void obfuscateQueries( if (token.type == TokenType::BareWord) { - if (keywords.count(whole_token) + std::string whole_token_uppercase(whole_token); + Poco::toUpperInPlace(whole_token_uppercase); + + if (keywords.count(whole_token_uppercase) || known_identifier_func(whole_token)) { /// Keep keywords as is. From 11ab250a712c686b6fee7ea14df60d2c57634bf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 05:18:18 +0300 Subject: [PATCH 120/131] More test cases --- tests/queries/0_stateless/01508_query_obfuscator.reference | 3 ++- tests/queries/0_stateless/01508_query_obfuscator.sh | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference index e2f61cf345d..7066528c870 100644 --- a/tests/queries/0_stateless/01508_query_obfuscator.reference +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -1 +1,2 @@ -SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Meantime_id_brisketHeavy, exclamation(Grit), avgIf(remote('128.0.0.1')) +SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1')) +SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh index 797271edb30..a5dd30b67ba 100755 --- a/tests/queries/0_stateless/01508_query_obfuscator.sh +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -4,3 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end" + From 5e146eda0403a781f78191f11b2a4f835d0b1307 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 06:02:05 +0300 Subject: [PATCH 121/131] Fix error --- src/Common/BitHelpers.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/BitHelpers.h b/src/Common/BitHelpers.h index eac5fdac80e..e79daeba14e 100644 --- a/src/Common/BitHelpers.h +++ b/src/Common/BitHelpers.h @@ -66,7 +66,7 @@ inline size_t getLeadingZeroBits(T x) template inline uint32_t bitScanReverse(T x) { - return sizeof(T) * 8 - 1 - getLeadingZeroBitsUnsafe(x); + return (std::max(sizeof(T), sizeof(unsigned int))) * 8 - 1 - getLeadingZeroBitsUnsafe(x); } // Unsafe since __builtin_ctz()-family explicitly state that result is undefined on x == 0 From 6fade0f3638af03377e93258486479a3c68bf025 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 00:47:54 -0300 Subject: [PATCH 122/131] stabilize merges --- tests/queries/0_stateless/01508_partition_pruning.queries | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 05f46e8e496..6c4698c482d 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -12,9 +12,11 @@ INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FRO INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +SYSTEM STOP MERGES tDD; insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +SYSTEM STOP MERGES sDD; insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); @@ -69,6 +71,7 @@ select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +SYSTEM START MERGES tDD OPTIMIZE TABLE tDD FINAL select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') From 57b480e65d5713198e0bd81e8cf84564d4e7d1b0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 07:03:55 +0300 Subject: [PATCH 123/131] Fix PVS warning --- programs/client/Suggest.cpp | 2 +- src/Parsers/obfuscateQueries.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index ac18a131c3a..e85e7a21261 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -80,7 +80,7 @@ Suggest::Suggest() "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", - "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE"}; } diff --git a/src/Parsers/obfuscateQueries.cpp b/src/Parsers/obfuscateQueries.cpp index 3594bdef3eb..32382b70bd7 100644 --- a/src/Parsers/obfuscateQueries.cpp +++ b/src/Parsers/obfuscateQueries.cpp @@ -37,7 +37,7 @@ const std::unordered_set keywords "WITH", "TOTALS", "HAVING", "ORDER", "COLLATE", "LIMIT", "UNION", "AND", "OR", "ASC", "IN", "KILL", "QUERY", "SYNC", "ASYNC", "TEST", "BETWEEN", "TRUNCATE", "USER", "ROLE", "PROFILE", "QUOTA", "POLICY", "ROW", "GRANT", "REVOKE", "OPTION", "ADMIN", "EXCEPT", "REPLACE", - "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "FOR", "RANDOMIZED", + "IDENTIFIED", "HOST", "NAME", "READONLY", "WRITABLE", "PERMISSIVE", "FOR", "RESTRICTIVE", "RANDOMIZED", "INTERVAL", "LIMITS", "ONLY", "TRACKING", "IP", "REGEXP", "ILIKE", "DICTIONARY" }; From da8a93806921085fdd6fda50fdf2db84f53e74e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 26 Sep 2020 09:50:34 +0300 Subject: [PATCH 124/131] Replace unit test to functional test --- src/CMakeLists.txt | 2 +- src/Parsers/tests/gtest_obfuscate_queries.cpp | 92 ------------------- .../01508_query_obfuscator.reference | 14 +++ .../0_stateless/01508_query_obfuscator.sh | 14 +++ 4 files changed, 29 insertions(+), 93 deletions(-) delete mode 100644 src/Parsers/tests/gtest_obfuscate_queries.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 40815228eac..0016c51b7f8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -415,6 +415,6 @@ if (ENABLE_TESTS AND USE_GTEST) -Wno-gnu-zero-variadic-macro-arguments ) - target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_table_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) + target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_aggregate_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils) add_check(unit_tests_dbms) endif () diff --git a/src/Parsers/tests/gtest_obfuscate_queries.cpp b/src/Parsers/tests/gtest_obfuscate_queries.cpp deleted file mode 100644 index ada15109d17..00000000000 --- a/src/Parsers/tests/gtest_obfuscate_queries.cpp +++ /dev/null @@ -1,92 +0,0 @@ -#include -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -using namespace DB; - - -TEST(ObfuscateQueries, Test1) -{ - WordMap obfuscated_words_map; - WordSet used_nouns; - SipHash hash_func; - - std::string salt = "Hello, world"; - hash_func.update(salt); - - SharedContextHolder shared_context; - const ContextHolder & context_holder = getContext(); - - registerFunctions(); - registerAggregateFunctions(); - registerTableFunctions(); - registerStorages(); - - std::unordered_set additional_names; - - auto all_known_storage_names = StorageFactory::instance().getAllRegisteredNames(); - auto all_known_data_type_names = DataTypeFactory::instance().getAllRegisteredNames(); - - additional_names.insert(all_known_storage_names.begin(), all_known_storage_names.end()); - additional_names.insert(all_known_data_type_names.begin(), all_known_data_type_names.end()); - - KnownIdentifierFunc is_known_identifier = [&](std::string_view name) - { - std::string what(name); - - return FunctionFactory::instance().tryGet(what, context_holder.context) != nullptr - || AggregateFunctionFactory::instance().isAggregateFunctionName(what) - || TableFunctionFactory::instance().isTableFunctionName(what) - || additional_names.count(what); - }; - - WriteBufferFromOwnString out; - - obfuscateQueries( - R"( -SELECT - VisitID, - Goals.ID, Goals.EventTime, - WatchIDs, - EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type -FROM merge.visits_v2 -WHERE - StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' - AND CounterID = 24226447 - AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 - AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) - AND Sign = 1 -)", - out, obfuscated_words_map, used_nouns, hash_func, is_known_identifier); - - EXPECT_EQ(out.str(), R"( -SELECT - CorduroyID, - Steel.ID, Steel.AcornSidestream, - WealthBRANCH, - GOVERNMENT.SedimentName, GOVERNMENT.SedimentExhaustion, GOVERNMENT.SedimentFencing, GOVERNMENT.SedimentOpossum, GOVERNMENT.AcornSidestream, GOVERNMENT.Lute -FROM merge.luncheonette_pants -WHERE - GovernanceCreche >= '2021-04-16' AND GovernanceCreche <= '2021-04-24' - AND StarboardID = 26446940 - AND intHash32(MessyID) = 474525514 AND intHash64(MessyID) = 13916317227779800149 - AND CorduroyID IN (5223158832904664474, 5605365157729463108, 7543250143731591192, 8715842063486405567, 7837015536326316923) - AND Tea = 1 -)"); -} - diff --git a/tests/queries/0_stateless/01508_query_obfuscator.reference b/tests/queries/0_stateless/01508_query_obfuscator.reference index 7066528c870..0064ac73a09 100644 --- a/tests/queries/0_stateless/01508_query_obfuscator.reference +++ b/tests/queries/0_stateless/01508_query_obfuscator.reference @@ -1,2 +1,16 @@ SELECT 116, 'Qqfu://2020-02-10isqkc1203 sp 2000-05-27T18:38:01', 13e100, Residue_id_breakfastDevice, park(Innervation), avgIf(remote('128.0.0.1')) SELECT shell_dust_tintype between crumb and shoat, case when peach >= 116 then bombing else null end + +SELECT + ChimeID, + Testimonial.ID, Testimonial.SipCauseway, + TankfulTRUMPET, + HUMIDITY.TermiteName, HUMIDITY.TermiteSculptural, HUMIDITY.TermiteGuilt, HUMIDITY.TermiteIntensity, HUMIDITY.SipCauseway, HUMIDITY.Coat +FROM merge.tinkle_efficiency +WHERE + FaithSeller >= '2020-10-13' AND FaithSeller <= '2020-10-21' + AND MandolinID = 30750384 + AND intHash32(GafferID) = 448362928 AND intHash64(GafferID) = 12572659331310383983 + AND ChimeID IN (8195672321757027078, 7079643623150622129, 5057006826979676478, 7886875230160484653, 7494974311229040743) + AND Stot = 1 + diff --git a/tests/queries/0_stateless/01508_query_obfuscator.sh b/tests/queries/0_stateless/01508_query_obfuscator.sh index a5dd30b67ba..d60e42489fa 100755 --- a/tests/queries/0_stateless/01508_query_obfuscator.sh +++ b/tests/queries/0_stateless/01508_query_obfuscator.sh @@ -6,3 +6,17 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT 123, 'Test://2020-01-01hello1234 at 2000-01-01T01:02:03', 12e100, Gibberish_id_testCool, hello(World), avgIf(remote('127.0.0.1'))" $CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< "SELECT cost_first_screen between a and b, case when x >= 123 then y else null end" +$CLICKHOUSE_FORMAT --seed Hello --obfuscate <<< " +SELECT + VisitID, + Goals.ID, Goals.EventTime, + WatchIDs, + EAction.ProductName, EAction.ProductPrice, EAction.ProductCurrency, EAction.ProductQuantity, EAction.EventTime, EAction.Type +FROM merge.visits_v2 +WHERE + StartDate >= '2020-09-17' AND StartDate <= '2020-09-25' + AND CounterID = 24226447 + AND intHash32(UserID) = 416638616 AND intHash64(UserID) = 13269091395366875299 + AND VisitID IN (5653048135597886819, 5556254872710352304, 5516214175671455313, 5476714937521999313, 5464051549483503043) + AND Sign = 1 +" From d1bb6b655553d429dd47dbc2079f8de26a4a238c Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 26 Sep 2020 11:47:26 +0300 Subject: [PATCH 125/131] Update test.py --- tests/integration/test_s3_with_proxy/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 3d118266455..70a50ae0e15 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -40,7 +40,7 @@ def cluster(): def check_proxy_logs(cluster, proxy_instance, http_methods={"POST", "PUT", "GET", "DELETE"}): for i in range(10): logs = cluster.get_container_logs(proxy_instance) - # Check that all possible interactions with Minio are present + # Check with retry that all possible interactions with Minio are present for http_method in http_methods: if logs.find(http_method + " http://minio1") >= 0: return From 44846da2b18c5cadbe20368d723f49f8326a9528 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 12:21:48 -0300 Subject: [PATCH 126/131] added test for queries results --- .../01508_partition_pruning.queries | 176 +++++++++--------- .../01508_partition_pruning.reference | 3 + ..._partition_pruning_check_results.reference | 60 ++++++ .../01508_partition_pruning_check_results.sql | 121 ++++++++++++ 4 files changed, 272 insertions(+), 88 deletions(-) create mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.reference create mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.sql diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 6c4698c482d..4fbe97a0f90 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -1,21 +1,21 @@ -DROP TABLE IF EXISTS tMM -DROP TABLE IF EXISTS tDD -DROP TABLE IF EXISTS sDD -DROP TABLE IF EXISTS xMM -CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() +DROP TABLE IF EXISTS tMM; +DROP TABLE IF EXISTS tDD; +DROP TABLE IF EXISTS sDD; +DROP TABLE IF EXISTS xMM; +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES tMM; -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000) -INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000) +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); -CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES tDD; insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); -CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES sDD; insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); @@ -24,98 +24,98 @@ insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); -CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; SYSTEM STOP MERGES xMM; -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000) -INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000) +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; -SYSTEM START MERGES tMM -OPTIMIZE TABLE tMM FINAL +SYSTEM START MERGES tMM; +OPTIMIZE TABLE tMM FINAL; -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; -SYSTEM START MERGES tDD -OPTIMIZE TABLE tDD FINAL +SYSTEM START MERGES tDD; +OPTIMIZE TABLE tDD FINAL; -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' -select uniqExact(_part), count() from sDD where d >= 1598918400000 -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +select uniqExact(_part), count() from sDD; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where d >= 1598918400000; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 -select uniqExact(_part), count() from xMM where a = 1 -select uniqExact(_part), count() from xMM where a = 66 -select uniqExact(_part), count() from xMM where a <> 66 -select uniqExact(_part), count() from xMM where a = 2 +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where a = 66; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where a = 2; SYSTEM START MERGES xMM; optimize table xMM final; -select uniqExact(_part), count() from xMM where a = 1 -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' -select uniqExact(_part), count() from xMM where a <> 66 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 - -DROP TABLE tMM -DROP TABLE tDD -DROP TABLE sDD -DROP TABLE xMM +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +DROP TABLE tMM; +DROP TABLE tDD; +DROP TABLE sDD; +DROP TABLE xMM; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index d967c760e91..58ba0c38106 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -106,6 +106,9 @@ Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary ke select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges +select uniqExact(_part), count() from sDD; +Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges + select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference new file mode 100644 index 00000000000..52182ee1b6f --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference @@ -0,0 +1,60 @@ +0 0 +2 2880 +1 1440 +0 0 +2 10000 +2 2880 +1 1440 +1 1440 +3 15000 +6 30000 +0 0 +2 6440 +2 10000 +2 2880 +1 1440 +2 6440 +4 20000 +2 10000 +1 1440 +3 11440 +1 1440 +3 11440 +1 1440 +2 10000 +3 9999 +2 10000 +4 20000 +2 10000 +2 9999 +1 10000 +2 20000 +1 10000 +1 10000 +1 10000 +3 40000 +3 40000 +6 30000 +3 9999 +2 9999 +0 0 +3 11440 +2 10000 +4 20000 +3 10001 +2 10000 +3 10001 +2 10000 +1 1 +2 5001 +1 5000 +2 10000 +3 15000 +0 0 +6 30000 +2 10000 +2 15000 +1 10000 +5 30000 +2 5001 +1 5000 diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql new file mode 100644 index 00000000000..4fbe97a0f90 --- /dev/null +++ b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql @@ -0,0 +1,121 @@ +DROP TABLE IF EXISTS tMM; +DROP TABLE IF EXISTS tDD; +DROP TABLE IF EXISTS sDD; +DROP TABLE IF EXISTS xMM; +CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES tMM; +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); +INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); + +CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES tDD; +insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); + +CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES sDD; +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); +insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); +insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); + +CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; +SYSTEM STOP MERGES xMM; +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); +INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); + + + +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; + +SYSTEM START MERGES tMM; +OPTIMIZE TABLE tMM FINAL; + +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; + +SYSTEM START MERGES tDD; +OPTIMIZE TABLE tDD FINAL; + +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; + + +select uniqExact(_part), count() from sDD; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where d >= 1598918400000; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; + + + +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where a = 66; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where a = 2; + +SYSTEM START MERGES xMM; +optimize table xMM final; + +select uniqExact(_part), count() from xMM where a = 1; +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +select uniqExact(_part), count() from xMM where a <> 66; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; + +DROP TABLE tMM; +DROP TABLE tDD; +DROP TABLE sDD; +DROP TABLE xMM; + + From 06f3991b745df55df3fc2196b322dd52cdb42612 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 12:25:47 -0300 Subject: [PATCH 127/131] added test for queries results --- .../01508_partition_pruning.reference | 118 +++++++++--------- 1 file changed, 59 insertions(+), 59 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index 58ba0c38106..bf68bd78cb9 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -1,180 +1,180 @@ -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15') +select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15' +select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015 +select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15' +select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00') +select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00' +select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01' +select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 +select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24') +select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24') +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26' +select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges select uniqExact(_part), count() from sDD; Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110 +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00' +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 +select uniqExact(_part), count() from sDD where d >= 1598918400000; Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010 +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 1; Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from xMM where a = 66 +select uniqExact(_part), count() from xMM where a = 66; Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a <> 66; Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from xMM where a = 2 +select uniqExact(_part), count() from xMM where a = 2; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where a = 1 +select uniqExact(_part), count() from xMM where a = 1; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00' +select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges -select uniqExact(_part), count() from xMM where a <> 66 +select uniqExact(_part), count() from xMM where a <> 66; Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3 +select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges From cb901ac3045d818cb1a4609bc29f19ba481077d0 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 19:07:13 -0300 Subject: [PATCH 128/131] fix test (UTC timezone specified) --- .../01508_partition_pruning.queries | 19 +-- .../01508_partition_pruning.reference | 76 ++++++++++- .../0_stateless/01508_partition_pruning.sh | 3 +- ..._partition_pruning_check_results.reference | 60 --------- .../01508_partition_pruning_check_results.sql | 121 ------------------ 5 files changed, 83 insertions(+), 196 deletions(-) delete mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.reference delete mode 100644 tests/queries/0_stateless/01508_partition_pruning_check_results.sql diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index 4fbe97a0f90..a30b541c3bc 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -34,7 +34,7 @@ INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); - +SELECT '--------- tMM ----------------------------'; select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); @@ -71,6 +71,8 @@ select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; + +SELECT '--------- tDD ----------------------------'; SYSTEM START MERGES tDD; OPTIMIZE TABLE tDD FINAL; @@ -81,17 +83,18 @@ select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and to select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; +SELECT '--------- sDD ----------------------------'; select uniqExact(_part), count() from sDD; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; select uniqExact(_part), count() from sDD where d >= 1598918400000; -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; - +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; +SELECT '--------- xMM ----------------------------'; select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index bf68bd78cb9..97a27f6fa8b 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -1,180 +1,244 @@ +--------- tMM ---------------------------- select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; +3 15000 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; +2 6440 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; +2 2880 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; +2 6440 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; +1 1440 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +3 9999 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; +2 9999 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; +2 20000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges +--------- tDD ---------------------------- select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; +3 40000 Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; +3 40000 Selected 3 parts by partition key, 3 parts by primary key, 4 marks by primary key, 4 marks to read from 3 ranges +--------- sDD ---------------------------- select uniqExact(_part), count() from sDD; +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; +3 9999 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; +2 9999 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from sDD where d >= 1598918400000; +4 20000 Selected 4 parts by partition key, 4 parts by primary key, 4 marks by primary key, 4 marks to read from 4 ranges -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; +select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; +3 10001 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges +--------- xMM ---------------------------- select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; +3 10001 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; +1 1 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where a = 1; +3 15000 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges select uniqExact(_part), count() from xMM where a = 66; +0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges select uniqExact(_part), count() from xMM where a <> 66; +6 30000 Selected 6 parts by partition key, 6 parts by primary key, 6 marks by primary key, 6 marks to read from 6 ranges select uniqExact(_part), count() from xMM where a = 2; +2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where a = 1; +2 15000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; +1 10000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges select uniqExact(_part), count() from xMM where a <> 66; +5 30000 Selected 5 parts by partition key, 5 parts by primary key, 5 marks by primary key, 5 marks to read from 5 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; +2 5001 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; +1 5000 Selected 1 parts by partition key, 1 parts by primary key, 1 marks by primary key, 1 marks to read from 1 ranges diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index e06b907f19f..e155431380f 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -20,8 +20,9 @@ do [ -z "$sql" ] && continue if [[ "$sql" == select* ]] ; then - CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') echo "$sql" + ${CLICKHOUSE_CLIENT} --query "$sql" + CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g') ${CLICKHOUSE_CLIENT} --query "$sql" 2>&1 | grep -oh "Selected .* parts by partition key, *. parts by primary key, .* marks by primary key, .* marks to read from .* ranges.*$" CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g') echo "" diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference b/tests/queries/0_stateless/01508_partition_pruning_check_results.reference deleted file mode 100644 index 52182ee1b6f..00000000000 --- a/tests/queries/0_stateless/01508_partition_pruning_check_results.reference +++ /dev/null @@ -1,60 +0,0 @@ -0 0 -2 2880 -1 1440 -0 0 -2 10000 -2 2880 -1 1440 -1 1440 -3 15000 -6 30000 -0 0 -2 6440 -2 10000 -2 2880 -1 1440 -2 6440 -4 20000 -2 10000 -1 1440 -3 11440 -1 1440 -3 11440 -1 1440 -2 10000 -3 9999 -2 10000 -4 20000 -2 10000 -2 9999 -1 10000 -2 20000 -1 10000 -1 10000 -1 10000 -3 40000 -3 40000 -6 30000 -3 9999 -2 9999 -0 0 -3 11440 -2 10000 -4 20000 -3 10001 -2 10000 -3 10001 -2 10000 -1 1 -2 5001 -1 5000 -2 10000 -3 15000 -0 0 -6 30000 -2 10000 -2 15000 -1 10000 -5 30000 -2 5001 -1 5000 diff --git a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql b/tests/queries/0_stateless/01508_partition_pruning_check_results.sql deleted file mode 100644 index 4fbe97a0f90..00000000000 --- a/tests/queries/0_stateless/01508_partition_pruning_check_results.sql +++ /dev/null @@ -1,121 +0,0 @@ -DROP TABLE IF EXISTS tMM; -DROP TABLE IF EXISTS tDD; -DROP TABLE IF EXISTS sDD; -DROP TABLE IF EXISTS xMM; -CREATE TABLE tMM(d DateTime,a Int64) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES tMM; -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, number FROM numbers(5000); -INSERT INTO tMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, number FROM numbers(5000); - -CREATE TABLE tDD(d DateTime,a Int) ENGINE = MergeTree PARTITION BY toYYYYMMDD(d) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES tDD; -insert into tDD select toDateTime(toDate('2020-09-23')), number from numbers(10000) UNION ALL select toDateTime(toDateTime('2020-09-23 11:00:00')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-24')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-09-25')), number from numbers(10000) UNION ALL select toDateTime(toDate('2020-08-15')), number from numbers(10000); - -CREATE TABLE sDD(d UInt64,a Int) ENGINE = MergeTree PARTITION BY toYYYYMM(toDate(intDiv(d,1000))) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES sDD; -insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); -insert into sDD select (1597536000+number*60)*1000, number from numbers(5000); -insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); -insert into sDD select (1598918400+number*60)*1000, number from numbers(5000); -insert into sDD select (1601510400+number*60)*1000, number from numbers(5000); -insert into sDD select (1602720000+number*60)*1000, number from numbers(5000); - -CREATE TABLE xMM(d DateTime,a Int64, f Int64) ENGINE = MergeTree PARTITION BY (toYYYYMM(d), a) ORDER BY tuple() SETTINGS index_granularity = 8192; -SYSTEM STOP MERGES xMM; -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 1, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-08-16 00:00:00') + number*60, 2, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 3, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-09-01 00:00:00') + number*60, 2, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-10-01 00:00:00') + number*60, 1, number FROM numbers(5000); -INSERT INTO xMM SELECT toDateTime('2020-10-15 00:00:00') + number*60, 1, number FROM numbers(5000); - - - -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-15'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-09-01'); -select uniqExact(_part), count() from tMM where toDate(d)=toDate('2020-10-15'); -select uniqExact(_part), count() from tMM where toDate(d)='2020-09-15'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)=202009; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20200816; -select uniqExact(_part), count() from tMM where toYYYYMMDD(d)=20201015; -select uniqExact(_part), count() from tMM where toDate(d)='2020-10-15'; -select uniqExact(_part), count() from tMM where d >= '2020-09-01 00:00:00' and d<'2020-10-15 00:00:00'; -select uniqExact(_part), count() from tMM where d >= '2020-01-16 00:00:00' and d < toDateTime('2021-08-17 00:00:00'); -select uniqExact(_part), count() from tMM where d >= '2020-09-16 00:00:00' and d < toDateTime('2020-10-01 00:00:00'); -select uniqExact(_part), count() from tMM where d >= '2020-09-12 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-12 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-09-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toStartOfDay(d) >= '2020-09-15 00:00:00' and d < '2020-10-16 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202009; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010 and toStartOfDay(d) = '2020-10-01 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) >= 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d) > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202009 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010 and toStartOfDay(d) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-15'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01'; -select uniqExact(_part), count() from tMM where toStartOfMonth(d) >= '2020-09-01' and toStartOfMonth(d) < '2020-10-01'; - -SYSTEM START MERGES tMM; -OPTIMIZE TABLE tMM FINAL; - -select uniqExact(_part), count() from tMM where toYYYYMM(d-1)+1 = 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d)+1 > 202010; -select uniqExact(_part), count() from tMM where toYYYYMM(d) between 202009 and 202010; - -SYSTEM START MERGES tDD; -OPTIMIZE TABLE tDD FINAL; - -select uniqExact(_part), count() from tDD where toDate(d)=toDate('2020-09-24'); -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = toDate('2020-09-24'); -select uniqExact(_part), count() FROM tDD WHERE toDate(d) = '2020-09-24'; -select uniqExact(_part), count() FROM tDD WHERE toDate(d) >= '2020-09-23' and toDate(d) <= '2020-09-26'; -select uniqExact(_part), count() FROM tDD WHERE toYYYYMMDD(d) >= 20200923 and toDate(d) <= '2020-09-26'; - - -select uniqExact(_part), count() from sDD; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1)+1 = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202010; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000))-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000)))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; -select uniqExact(_part), count() from sDD where d >= 1598918400000; -select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000))-1) < 202010; - - - -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a=1; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-11-01 00:00:00' and a = 1; -select uniqExact(_part), count() from xMM where a = 1; -select uniqExact(_part), count() from xMM where a = 66; -select uniqExact(_part), count() from xMM where a <> 66; -select uniqExact(_part), count() from xMM where a = 2; - -SYSTEM START MERGES xMM; -optimize table xMM final; - -select uniqExact(_part), count() from xMM where a = 1; -select uniqExact(_part), count() from xMM where toStartOfDay(d) >= '2020-10-01 00:00:00'; -select uniqExact(_part), count() from xMM where a <> 66; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d <= '2020-10-01 00:00:00' and a<>3; -select uniqExact(_part), count() from xMM where d >= '2020-09-01 00:00:00' and d < '2020-10-01 00:00:00' and a<>3; - -DROP TABLE tMM; -DROP TABLE tDD; -DROP TABLE sDD; -DROP TABLE xMM; - - From a8618c96f798c36ae3ea07688dc35ff604d87b9c Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Sat, 26 Sep 2020 21:02:18 -0300 Subject: [PATCH 129/131] fix test (UTC timezone specified) more UTC --- tests/queries/0_stateless/01508_partition_pruning.queries | 4 ++-- tests/queries/0_stateless/01508_partition_pruning.reference | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.queries b/tests/queries/0_stateless/01508_partition_pruning.queries index a30b541c3bc..3773e907c53 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.queries +++ b/tests/queries/0_stateless/01508_partition_pruning.queries @@ -88,8 +88,8 @@ select uniqExact(_part), count() from sDD; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1)+1 = 202010; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202010; select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) = 202110; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); select uniqExact(_part), count() from sDD where d >= 1598918400000; select uniqExact(_part), count() from sDD where d >= 1598918400000 and toYYYYMM(toDateTime(intDiv(d,1000),'UTC')-1) < 202010; diff --git a/tests/queries/0_stateless/01508_partition_pruning.reference b/tests/queries/0_stateless/01508_partition_pruning.reference index 97a27f6fa8b..0cc40d23b41 100644 --- a/tests/queries/0_stateless/01508_partition_pruning.reference +++ b/tests/queries/0_stateless/01508_partition_pruning.reference @@ -161,11 +161,11 @@ select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,100 0 0 Selected 0 parts by partition key, 0 parts by primary key, 0 marks by primary key, 0 marks to read from 0 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000))) < '2020-10-02 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toStartOfDay(toDateTime(intDiv(d,1000),'UTC')) < toDateTime('2020-10-02 00:00:00','UTC'); 3 11440 Selected 3 parts by partition key, 3 parts by primary key, 3 marks by primary key, 3 marks to read from 3 ranges -select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000)) < '2020-10-01 00:00:00'; +select uniqExact(_part), count() from sDD where toYYYYMM(toDateTime(intDiv(d,1000),'UTC'))+1 > 202009 and toDateTime(intDiv(d,1000),'UTC') < toDateTime('2020-10-01 00:00:00','UTC'); 2 10000 Selected 2 parts by partition key, 2 parts by primary key, 2 marks by primary key, 2 marks to read from 2 ranges From 0dd244126d1213fd5cd913319f4a779b2b34d4e0 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 27 Sep 2020 10:03:13 -0300 Subject: [PATCH 130/131] Update 01508_partition_pruning.sh --- tests/queries/0_stateless/01508_partition_pruning.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01508_partition_pruning.sh b/tests/queries/0_stateless/01508_partition_pruning.sh index e155431380f..c886946c7d9 100755 --- a/tests/queries/0_stateless/01508_partition_pruning.sh +++ b/tests/queries/0_stateless/01508_partition_pruning.sh @@ -1,10 +1,13 @@ #!/usr/bin/env bash -#-------------------------------------------- +#------------------------------------------------------------------------------------------- # Description of test result: # Test the correctness of the partition # pruning -#-------------------------------------------- +# +# Script executes queries from a file 01508_partition_pruning.queries (1 line = 1 query) +# Queries are started with 'select' (but NOT with 'SELECT') are executed with log_level=debug +#------------------------------------------------------------------------------------------- CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh From 07b931a41473481f7a4978109b48ec9610c25877 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 28 Sep 2020 04:21:40 +0300 Subject: [PATCH 131/131] Prepare for LLVM-11 --- CMakeLists.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index be1b6ac04f5..f4e230fbd93 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -173,7 +173,7 @@ endif () # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") -find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") +find_program (OBJCOPY_PATH NAMES "llvm-objcopy" "llvm-objcopy-11" "llvm-objcopy-10" "llvm-objcopy-9" "llvm-objcopy-8" "objcopy") if (OBJCOPY_PATH) message(STATUS "Using objcopy: ${OBJCOPY_PATH}.") @@ -313,7 +313,7 @@ if (COMPILER_CLANG) endif () # Always prefer llvm tools when using clang. For instance, we cannot use GNU ar when llvm LTO is enabled - find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") + find_program (LLVM_AR_PATH NAMES "llvm-ar" "llvm-ar-11" "llvm-ar-10" "llvm-ar-9" "llvm-ar-8") if (LLVM_AR_PATH) message(STATUS "Using llvm-ar: ${LLVM_AR_PATH}.") @@ -322,7 +322,7 @@ if (COMPILER_CLANG) message(WARNING "Cannot find llvm-ar. System ar will be used instead. It does not work with ThinLTO.") endif () - find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") + find_program (LLVM_RANLIB_PATH NAMES "llvm-ranlib" "llvm-ranlib-11" "llvm-ranlib-10" "llvm-ranlib-9" "llvm-ranlib-8") if (LLVM_RANLIB_PATH) message(STATUS "Using llvm-ranlib: ${LLVM_RANLIB_PATH}.")