From b33a5e0d140dc9c9a942d6bc05c02d292143ae80 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 16 Sep 2020 15:32:33 +0300 Subject: [PATCH 01/64] 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 02/64] 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 03/64] 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 04/64] 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 05/64] 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 06/64] 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 07/64] 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 08/64] 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 09/64] 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 10/64] 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 11/64] 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 12/64] 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 13/64] 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 14/64] 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 15/64] 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 d87f3660ee2d75c993bc2b6356dae37a9186e6e3 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Mon, 21 Sep 2020 23:19:46 +0300 Subject: [PATCH 16/64] 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 d7c7619cd84bea7b6bc8c63b98911e29446c8732 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 22 Sep 2020 13:13:57 +0300 Subject: [PATCH 17/64] 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 18/64] 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 cf24789c1b6fe1dc81443cbf1ce6b4b0f10dd97d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 22 Sep 2020 21:25:31 +0300 Subject: [PATCH 19/64] 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 2d4bdcfec0e2982f102a72b6f7fd40acd197e421 Mon Sep 17 00:00:00 2001 From: nikitamikhaylov Date: Wed, 23 Sep 2020 22:31:47 +0300 Subject: [PATCH 20/64] 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 21/64] 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 22/64] 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 23/64] 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 a85bb16c66f0c795290cf8dc740c32c4d3ba0493 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 10:33:22 +0300 Subject: [PATCH 24/64] 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 25/64] 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 26/64] 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 27/64] 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 4f3eb4532ace4f133b1a680e424e88e63438cb1a Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 11:37:20 +0300 Subject: [PATCH 28/64] 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 29/64] 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 b8861ad6cc00cd60330f253dac999053dde8ce92 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 24 Sep 2020 13:21:34 +0300 Subject: [PATCH 30/64] 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 31/64] 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 32/64] 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 33/64] 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 34/64] 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 35/64] 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 36/64] Bump CI. 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 37/64] 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 38/64] 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 39/64] 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 40/64] 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 651518566a527f8a0c225f1d8a130712c8b810ee Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 24 Sep 2020 16:24:44 +0300 Subject: [PATCH 41/64] 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 42/64] 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 43/64] 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 a4a5b96d4a0915ca74c41b9c8ed7952c46a1e150 Mon Sep 17 00:00:00 2001 From: alekseik1 <1alekseik1@gmail.com> Date: Thu, 24 Sep 2020 20:08:16 +0300 Subject: [PATCH 44/64] 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 45/64] 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 46/64] 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 47/64] 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 48/64] 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 49/64] 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 50/64] 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 51/64] 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 52/64] 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 53/64] 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 54/64] 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 55/64] 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 56/64] 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 57/64] 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 58/64] 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 59/64] 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 60/64] 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 61/64] 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 62/64] 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 63/64] 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 23f06de0bef472ccf18b950c2afc7629c7266a65 Mon Sep 17 00:00:00 2001 From: John <108540613@qq.com> Date: Fri, 25 Sep 2020 11:48:25 +0800 Subject: [PATCH 64/64] 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` 运算符是单独复盖的,因为它们的功能相当丰富。